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

[01/49] carbondata git commit: [CARBONDATA-1992] Remove partitionId in CarbonTablePath

Repository: carbondata
Updated Branches:
  refs/heads/carbonstore-rebase4 [created] 7466d6538


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
index d321405..58009af 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
@@ -65,21 +66,21 @@ public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
     child.initialize();
   }
 
-  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier, String partitionId) {
+  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier) {
     String[] storeLocation = CarbonDataProcessorUtil
         .getLocalDataFolderLocation(tableIdentifier.getDatabaseName(),
-            tableIdentifier.getTableName(), String.valueOf(configuration.getTaskNo()), partitionId,
-            configuration.getSegmentId() + "", false, false);
+            tableIdentifier.getTableName(), String.valueOf(configuration.getTaskNo()),
+            configuration.getSegmentId(), false, false);
     CarbonDataProcessorUtil.createLocations(storeLocation);
     return storeLocation;
   }
 
-  public CarbonFactDataHandlerModel getDataHandlerModel(int partitionId) {
+  public CarbonFactDataHandlerModel getDataHandlerModel() {
     CarbonTableIdentifier tableIdentifier =
         configuration.getTableIdentifier().getCarbonTableIdentifier();
-    String[] storeLocation = getStoreLocation(tableIdentifier, String.valueOf(partitionId));
+    String[] storeLocation = getStoreLocation(tableIdentifier);
     return CarbonFactDataHandlerModel.createCarbonFactDataHandlerModel(configuration,
-        storeLocation, partitionId, 0);
+        storeLocation, 0, 0);
   }
 
   @Override public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
@@ -89,11 +90,11 @@ public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
     String tableName = tableIdentifier.getTableName();
     try {
       CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+          .recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
               System.currentTimeMillis());
       int i = 0;
       for (Iterator<CarbonRowBatch> iterator : iterators) {
-        String[] storeLocation = getStoreLocation(tableIdentifier, String.valueOf(i));
+        String[] storeLocation = getStoreLocation(tableIdentifier);
 
         CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
             .createCarbonFactDataHandlerModel(configuration, storeLocation, i, 0);
@@ -147,10 +148,11 @@ public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
     processingComplete(dataHandler);
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+        .recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
             System.currentTimeMillis());
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
+        .recordMdkGenerateTotalTime(CarbonTablePath.DEPRECATED_PATITION_ID,
+            System.currentTimeMillis());
   }
 
   private void processingComplete(CarbonFactHandler dataHandler) throws CarbonDataLoadingException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index 3f1430d..c141636 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -321,7 +321,7 @@ public final class CarbonDataMergerUtil {
 
         // create entry for merged one.
         LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
-        loadMetadataDetails.setPartitionCount(carbonLoadModel.getPartitionId());
+        loadMetadataDetails.setPartitionCount(CarbonTablePath.DEPRECATED_PATITION_ID);
         loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS);
         long loadEnddate = CarbonUpdateUtil.readCurrentTime();
         loadMetadataDetails.setLoadEndTime(loadEnddate);
@@ -686,7 +686,7 @@ public final class CarbonDataMergerUtil {
       CarbonTableIdentifier carbonTableIdentifier, String segmentId) {
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier);
-    return carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
+    return carbonTablePath.getCarbonDataDirectoryPath(segmentId);
   }
 
 
@@ -1047,7 +1047,7 @@ public final class CarbonDataMergerUtil {
 
     CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
 
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath("0", seg.getSegmentNo());
+    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(seg.getSegmentNo());
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
     CarbonFile[] allSegmentFiles = segDir.listFiles();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index 2fbdf4f..bfe38fd 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -371,7 +371,7 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
     return SortParameters
         .createSortParameters(carbonTable, carbonLoadModel.getDatabaseName(), tableName,
             dimensionColumnCount, segmentProperties.getComplexDimensions().size(), measureCount,
-            noDictionaryCount, carbonLoadModel.getPartitionId(), segmentId,
+            noDictionaryCount, segmentId,
             carbonLoadModel.getTaskNo(), noDictionaryColMapping, true);
   }
 
@@ -431,7 +431,7 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
   private void initTempStoreLocation() {
     tempStoreLocation = CarbonDataProcessorUtil
         .getLocalDataFolderLocation(carbonLoadModel.getDatabaseName(), tableName,
-            carbonLoadModel.getTaskNo(), carbonLoadModel.getPartitionId(), segmentId,
+            carbonLoadModel.getTaskNo(), segmentId,
             true, false);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
index 98d150e..4d31f87 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
@@ -356,7 +357,7 @@ public class SortParameters implements Serializable {
     CarbonProperties carbonProperties = CarbonProperties.getInstance();
     parameters.setDatabaseName(tableIdentifier.getDatabaseName());
     parameters.setTableName(tableIdentifier.getTableName());
-    parameters.setPartitionID(configuration.getPartitionId());
+    parameters.setPartitionID("0");
     parameters.setSegmentId(configuration.getSegmentId());
     parameters.setTaskNo(configuration.getTaskNo());
     parameters.setMeasureColCount(configuration.getMeasureCount());
@@ -392,10 +393,9 @@ public class SortParameters implements Serializable {
 
     LOGGER.info("File Buffer Size: " + parameters.getFileBufferSize());
 
-    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(tableIdentifier.getDatabaseName(),
-            tableIdentifier.getTableName(), configuration.getTaskNo(),
-            configuration.getPartitionId(), configuration.getSegmentId(), false, false);
+    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil.getLocalDataFolderLocation(
+        tableIdentifier.getDatabaseName(), tableIdentifier.getTableName(),
+        configuration.getTaskNo(), configuration.getSegmentId(), false, false);
     String[] sortTempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
         File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
 
@@ -448,13 +448,13 @@ public class SortParameters implements Serializable {
 
   public static SortParameters createSortParameters(CarbonTable carbonTable, String databaseName,
       String tableName, int dimColCount, int complexDimColCount, int measureColCount,
-      int noDictionaryCount, String partitionID, String segmentId, String taskNo,
+      int noDictionaryCount, String segmentId, String taskNo,
       boolean[] noDictionaryColMaping, boolean isCompactionFlow) {
     SortParameters parameters = new SortParameters();
     CarbonProperties carbonProperties = CarbonProperties.getInstance();
     parameters.setDatabaseName(databaseName);
     parameters.setTableName(tableName);
-    parameters.setPartitionID(partitionID);
+    parameters.setPartitionID(CarbonTablePath.DEPRECATED_PATITION_ID);
     parameters.setSegmentId(segmentId);
     parameters.setTaskNo(taskNo);
     parameters.setMeasureColCount(measureColCount);
@@ -486,7 +486,7 @@ public class SortParameters implements Serializable {
     LOGGER.info("File Buffer Size: " + parameters.getFileBufferSize());
 
     String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(databaseName, tableName, taskNo, partitionID, segmentId,
+        .getLocalDataFolderLocation(databaseName, tableName, taskNo, segmentId,
             isCompactionFlow, false);
     String[] sortTempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
         File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index d77fcab..732a7e8 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -339,8 +339,7 @@ public class CarbonFactDataHandlerModel {
     AbsoluteTableIdentifier absoluteTableIdentifier = configuration.getTableIdentifier();
     CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     String carbonDataDirectoryPath = carbonTablePath
-        .getCarbonDataDirectoryPath(configuration.getPartitionId(),
-            configuration.getSegmentId() + "");
+        .getCarbonDataDirectoryPath(configuration.getSegmentId() + "");
     CarbonUtil.checkAndCreateFolder(carbonDataDirectoryPath);
     return carbonDataDirectoryPath;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index 1e648e1..e319160 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -123,13 +123,11 @@ public final class CarbonDataProcessorUtil {
    * @param databaseName
    * @param tableName
    * @param taskId
-   * @param partitionId
    * @param segmentId
    * @return
    */
   public static String[] getLocalDataFolderLocation(String databaseName, String tableName,
-      String taskId, String partitionId, String segmentId, boolean isCompactionFlow,
-      boolean isAltPartitionFlow) {
+      String taskId, String segmentId, boolean isCompactionFlow, boolean isAltPartitionFlow) {
     String tempLocationKey =
         getTempStoreLocationKey(databaseName, tableName, segmentId, taskId, isCompactionFlow,
             isAltPartitionFlow);
@@ -150,8 +148,7 @@ public final class CarbonDataProcessorUtil {
       String tmpStore = baseTmpStorePathArray[i];
       CarbonTablePath carbonTablePath =
           CarbonStorePath.getCarbonTablePath(tmpStore, carbonTable.getCarbonTableIdentifier());
-      String carbonDataDirectoryPath =
-          carbonTablePath.getCarbonDataDirectoryPath(partitionId, segmentId + "");
+      String carbonDataDirectoryPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
 
       localDataFolderLocArray[i] = carbonDataDirectoryPath + File.separator + taskId;
     }
@@ -385,7 +382,7 @@ public final class CarbonDataProcessorUtil {
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(factStoreLocation, carbonTableIdentifier);
     String carbonDataDirectoryPath =
-        carbonTablePath.getCarbonDataDirectoryPath(partitionId, segmentId);
+        carbonTablePath.getCarbonDataDirectoryPath(segmentId);
     return carbonDataDirectoryPath;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 32c72da..c135a88 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -78,10 +78,8 @@ public final class CarbonLoaderUtil {
     CarbonTablePath carbonTablePath = CarbonStorePath
         .getCarbonTablePath(loadModel.getTablePath(), carbonTable.getCarbonTableIdentifier());
 
-    for (int i = 0; i < carbonTable.getPartitionCount(); i++) {
-      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(i + "", currentLoad + "");
-      deleteStorePath(segmentPath);
-    }
+    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(currentLoad + "");
+    deleteStorePath(segmentPath);
   }
 
   /**
@@ -101,7 +99,7 @@ public final class CarbonLoaderUtil {
     int fileCount = 0;
     int partitionCount = carbonTable.getPartitionCount();
     for (int i = 0; i < partitionCount; i++) {
-      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(i + "",
+      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(
           currentLoad + "");
       CarbonFile carbonFile = FileFactory.getCarbonFile(segmentPath,
           FileFactory.getFileType(segmentPath));
@@ -330,7 +328,7 @@ public final class CarbonLoaderUtil {
 
   private static void addToStaleFolders(CarbonTablePath carbonTablePath,
       List<CarbonFile> staleFolders, LoadMetadataDetails entry) throws IOException {
-    String path = carbonTablePath.getCarbonDataDirectoryPath("0", entry.getLoadName());
+    String path = carbonTablePath.getCarbonDataDirectoryPath(entry.getLoadName());
     // add to the deletion list only if file exist else HDFS file system will throw
     // exception while deleting the file if file path does not exist
     if (FileFactory.isFileExist(path, FileFactory.getFileType(path))) {
@@ -911,7 +909,7 @@ public final class CarbonLoaderUtil {
     CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath(), carbonTableIdentifier);
-    String segmentFolder = carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
+    String segmentFolder = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
     CarbonUtil.checkAndCreateFolder(segmentFolder);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
index 52b9f52..288cd54 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
@@ -51,15 +51,14 @@ public final class DeleteLoadFolders {
    * returns segment path
    *
    * @param absoluteTableIdentifier
-   * @param partitionId
    * @param oneLoad
    * @return
    */
   private static String getSegmentPath(AbsoluteTableIdentifier absoluteTableIdentifier,
-      int partitionId, LoadMetadataDetails oneLoad) {
+      LoadMetadataDetails oneLoad) {
     CarbonTablePath carbon = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     String segmentId = oneLoad.getLoadName();
-    return carbon.getCarbonDataDirectoryPath("" + partitionId, segmentId);
+    return carbon.getCarbonDataDirectoryPath(segmentId);
   }
 
   public static void physicalFactAndMeasureMetadataDeletion(
@@ -74,7 +73,7 @@ public final class DeleteLoadFolders {
                 .deleteSegment(absoluteTableIdentifier.getTablePath(), oneLoad.getSegmentFile(),
                     specs);
           } else {
-            String path = getSegmentPath(absoluteTableIdentifier, 0, oneLoad);
+            String path = getSegmentPath(absoluteTableIdentifier, oneLoad);
             boolean status = false;
             if (FileFactory.isFileExist(path, FileFactory.getFileType(path))) {
               CarbonFile file = FileFactory.getCarbonFile(path, FileFactory.getFileType(path));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
index e662757..7f0aef6 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
@@ -163,7 +163,6 @@ public class StoreCreator {
       loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(","));
       loadModel.setTaskNo("0");
       loadModel.setSegmentId("0");
-      loadModel.setPartitionId("0");
       loadModel.setFactTimeStamp(System.currentTimeMillis());
       loadModel.setMaxColumns("10");
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
index 35a3513..897a812 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
@@ -108,7 +108,6 @@ class StreamHandoffRDD[K, V](
       split: Partition,
       context: TaskContext
   ): Iterator[(K, V)] = {
-    carbonLoadModel.setPartitionId("0")
     carbonLoadModel.setTaskNo("" + split.index)
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
     CarbonMetadata.getInstance().addCarbonTable(carbonTable)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
index 3366f51..75fcfb0 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -136,7 +136,7 @@ object StreamSinkFactory {
       FileFactory.mkdirs(carbonTablePath.getMetadataDirectoryPath, fileType)
     }
     val segmentId = StreamSegment.open(carbonTable)
-    val segmentDir = carbonTablePath.getSegmentDir("0", segmentId)
+    val segmentDir = carbonTablePath.getSegmentDir(segmentId)
     if (FileFactory.isFileExist(segmentDir, fileType)) {
       // recover fault
       StreamSegment.recoverSegmentIfRequired(segmentDir)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index 67d8a4d..206ba91 100644
--- a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -150,12 +150,12 @@ class CarbonAppendableStreamSink(
    * if the directory size of current segment beyond the threshold, hand off new segment
    */
   private def checkOrHandOffSegment(): Unit = {
-    val segmentDir = carbonTablePath.getSegmentDir("0", currentSegmentId)
+    val segmentDir = carbonTablePath.getSegmentDir(currentSegmentId)
     val fileType = FileFactory.getFileType(segmentDir)
     if (segmentMaxSize <= StreamSegment.size(segmentDir)) {
       val newSegmentId = StreamSegment.close(carbonTable, currentSegmentId)
       currentSegmentId = newSegmentId
-      val newSegmentDir = carbonTablePath.getSegmentDir("0", currentSegmentId)
+      val newSegmentDir = carbonTablePath.getSegmentDir(currentSegmentId)
       FileFactory.mkdirs(newSegmentDir, fileType)
 
       // TODO trigger hand off operation
@@ -252,14 +252,14 @@ object CarbonAppendableStreamSink {
 
         // update data file info in index file
         val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-        StreamSegment.updateIndexFile(tablePath.getSegmentDir("0", segmentId))
+        StreamSegment.updateIndexFile(tablePath.getSegmentDir(segmentId))
 
       } catch {
         // catch fault of executor side
         case t: Throwable =>
           val tablePath =
             CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-          val segmentDir = tablePath.getSegmentDir("0", segmentId)
+          val segmentDir = tablePath.getSegmentDir(segmentId)
           StreamSegment.recoverSegmentIfRequired(segmentDir)
           LOGGER.error(t, s"Aborting job ${ job.getJobID }.")
           committer.abortJob(job)


[14/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
index cc2e513..6e30021 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -43,10 +43,9 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TableBlockUniqueIdentifier;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNodeWrapper;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode;
 import org.apache.carbondata.core.indexstore.blockletindex.IndexWrapper;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -64,8 +63,8 @@ import org.apache.carbondata.core.scan.executor.util.RestructureUtil;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
 import org.apache.carbondata.core.scan.filter.TableProvider;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
@@ -121,7 +120,6 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     queryProperties.queryStatisticsRecorder =
         CarbonTimeStatisticsFactory.createExecutorRecorder(queryModel.getQueryId());
     queryModel.setStatisticsRecorder(queryProperties.queryStatisticsRecorder);
-    QueryUtil.resolveQueryModel(queryModel);
     QueryStatistic queryStatistic = new QueryStatistic();
     // sort the block info
     // so block will be loaded in sorted order this will be required for
@@ -168,12 +166,12 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
         .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR, System.currentTimeMillis());
     queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
     // calculating the total number of aggeragted columns
-    int measureCount = queryModel.getQueryMeasures().size();
+    int measureCount = queryModel.getProjectionMeasures().size();
 
     int currentIndex = 0;
     DataType[] dataTypes = new DataType[measureCount];
 
-    for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
+    for (ProjectionMeasure carbonMeasure : queryModel.getProjectionMeasures()) {
       // adding the data type and aggregation type of all the measure this
       // can be used
       // to select the aggregator
@@ -198,9 +196,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     queryStatistic = new QueryStatistic();
     // dictionary column unique column id to dictionary mapping
     // which will be used to get column actual data
-    queryProperties.columnToDictionayMapping = QueryUtil
-        .getDimensionDictionaryDetail(queryModel.getQueryDimension(),
-            queryProperties.complexFilterDimension, queryModel.getAbsoluteTableIdentifier(),
+    queryProperties.columnToDictionayMapping =
+        QueryUtil.getDimensionDictionaryDetail(
+            queryModel.getProjectionDimensions(),
+            queryProperties.complexFilterDimension,
+            queryModel.getAbsoluteTableIdentifier(),
             tableProvider);
     queryStatistic
         .addStatistics(QueryStatisticsConstants.LOAD_DICTIONARY, System.currentTimeMillis());
@@ -263,8 +263,8 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // and query will be executed based on that infos
     for (int i = 0; i < queryProperties.dataBlocks.size(); i++) {
       AbstractIndex abstractIndex = queryProperties.dataBlocks.get(i);
-      BlockletDataRefNodeWrapper dataRefNode =
-          (BlockletDataRefNodeWrapper) abstractIndex.getDataRefNode();
+      BlockletDataRefNode dataRefNode =
+          (BlockletDataRefNode) abstractIndex.getDataRefNode();
       blockExecutionInfoList.add(getBlockExecutionInfoForBlock(queryModel, abstractIndex,
           dataRefNode.getBlockInfos().get(0).getBlockletInfos().getStartBlockletNumber(),
           dataRefNode.numberOfNodes(), dataRefNode.getBlockInfos().get(0).getFilePath(),
@@ -289,47 +289,45 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
    * @return block execution info
    * @throws QueryExecutionException any failure during block info creation
    */
-  protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel,
+  private BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel,
       AbstractIndex blockIndex, int startBlockletIndex, int numberOfBlockletToScan, String filePath,
       String[] deleteDeltaFiles, String segmentId)
       throws QueryExecutionException {
     BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
     SegmentProperties segmentProperties = blockIndex.getSegmentProperties();
     List<CarbonDimension> tableBlockDimensions = segmentProperties.getDimensions();
-    KeyGenerator blockKeyGenerator = segmentProperties.getDimensionKeyGenerator();
 
     // below is to get only those dimension in query which is present in the
     // table block
-    List<QueryDimension> currentBlockQueryDimensions = RestructureUtil
+    List<ProjectionDimension> projectDimensions = RestructureUtil
         .createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo,
-            queryModel.getQueryDimension(), tableBlockDimensions,
+            queryModel.getProjectionDimensions(), tableBlockDimensions,
             segmentProperties.getComplexDimensions());
     blockExecutionInfo.setBlockId(
         CarbonUtil.getBlockId(queryModel.getAbsoluteTableIdentifier(), filePath, segmentId));
     blockExecutionInfo.setDeleteDeltaFilePath(deleteDeltaFiles);
     blockExecutionInfo.setStartBlockletIndex(startBlockletIndex);
     blockExecutionInfo.setNumberOfBlockletToScan(numberOfBlockletToScan);
-    blockExecutionInfo.setQueryDimensions(currentBlockQueryDimensions
-        .toArray(new QueryDimension[currentBlockQueryDimensions.size()]));
+    blockExecutionInfo.setProjectionDimensions(projectDimensions
+        .toArray(new ProjectionDimension[projectDimensions.size()]));
     // get measures present in the current block
-    List<QueryMeasure> currentBlockQueryMeasures =
+    List<ProjectionMeasure> currentBlockQueryMeasures =
         getCurrentBlockQueryMeasures(blockExecutionInfo, queryModel, blockIndex);
-    blockExecutionInfo.setQueryMeasures(
-        currentBlockQueryMeasures.toArray(new QueryMeasure[currentBlockQueryMeasures.size()]));
+    blockExecutionInfo.setProjectionMeasures(
+        currentBlockQueryMeasures.toArray(new ProjectionMeasure[currentBlockQueryMeasures.size()]));
     blockExecutionInfo.setDataBlock(blockIndex);
-    blockExecutionInfo.setBlockKeyGenerator(blockKeyGenerator);
     // setting whether raw record query or not
     blockExecutionInfo.setRawRecordDetailQuery(queryModel.isForcedDetailRawQuery());
     // total number dimension
     blockExecutionInfo
-        .setTotalNumberDimensionBlock(segmentProperties.getDimensionOrdinalToBlockMapping().size());
+        .setTotalNumberDimensionToRead(
+            segmentProperties.getDimensionOrdinalToChunkMapping().size());
     blockExecutionInfo.setPrefetchBlocklet(!queryModel.isReadPageByPage());
     blockExecutionInfo
-        .setTotalNumberOfMeasureBlock(segmentProperties.getMeasuresOrdinalToBlockMapping().size());
-    blockExecutionInfo.setAbsoluteTableIdentifier(queryModel.getAbsoluteTableIdentifier());
+        .setTotalNumberOfMeasureToRead(segmentProperties.getMeasuresOrdinalToChunkMapping().size());
     blockExecutionInfo.setComplexDimensionInfoMap(QueryUtil
-        .getComplexDimensionsMap(currentBlockQueryDimensions,
-            segmentProperties.getDimensionOrdinalToBlockMapping(),
+        .getComplexDimensionsMap(projectDimensions,
+            segmentProperties.getDimensionOrdinalToChunkMapping(),
             segmentProperties.getEachComplexDimColumnValueSize(),
             queryProperties.columnToDictionayMapping, queryProperties.complexFilterDimension));
     IndexKey startIndexKey = null;
@@ -363,45 +361,46 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // create a list of filter dimensions present in the current block
     Set<CarbonDimension> currentBlockFilterDimensions =
         getCurrentBlockFilterDimensions(queryProperties.complexFilterDimension, segmentProperties);
-    int[] dimensionsBlockIndexes = QueryUtil.getDimensionsBlockIndexes(currentBlockQueryDimensions,
-        segmentProperties.getDimensionOrdinalToBlockMapping(), expressionDimensions,
-        currentBlockFilterDimensions, allProjectionListDimensionIdexes);
+    int[] dimensionChunkIndexes = QueryUtil.getDimensionChunkIndexes(
+        projectDimensions, segmentProperties.getDimensionOrdinalToChunkMapping(),
+        expressionDimensions, currentBlockFilterDimensions, allProjectionListDimensionIdexes);
     int numberOfColumnToBeReadInOneIO = Integer.parseInt(CarbonProperties.getInstance()
         .getProperty(CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO,
             CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULTVALUE));
 
-    if (dimensionsBlockIndexes.length > 0) {
-      numberOfElementToConsider = dimensionsBlockIndexes[dimensionsBlockIndexes.length - 1]
+    if (dimensionChunkIndexes.length > 0) {
+      numberOfElementToConsider = dimensionChunkIndexes[dimensionChunkIndexes.length - 1]
           == segmentProperties.getBlockTodimensionOrdinalMapping().size() - 1 ?
-          dimensionsBlockIndexes.length - 1 :
-          dimensionsBlockIndexes.length;
-      blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(CarbonUtil
-          .getRangeIndex(dimensionsBlockIndexes, numberOfElementToConsider,
+          dimensionChunkIndexes.length - 1 :
+          dimensionChunkIndexes.length;
+      blockExecutionInfo.setAllSelectedDimensionColumnIndexRange(
+          CarbonUtil.getRangeIndex(dimensionChunkIndexes, numberOfElementToConsider,
               numberOfColumnToBeReadInOneIO));
     } else {
-      blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(new int[0][0]);
+      blockExecutionInfo.setAllSelectedDimensionColumnIndexRange(new int[0][0]);
     }
     // get the list of updated filter measures present in the current block
-    Set<CarbonMeasure> currentBlockFilterMeasures =
+    Set<CarbonMeasure> filterMeasures =
         getCurrentBlockFilterMeasures(queryProperties.filterMeasures, segmentProperties);
     // list of measures to be projected
     List<Integer> allProjectionListMeasureIndexes = new ArrayList<>();
-    int[] measureBlockIndexes = QueryUtil
-        .getMeasureBlockIndexes(currentBlockQueryMeasures, expressionMeasures,
-            segmentProperties.getMeasuresOrdinalToBlockMapping(), currentBlockFilterMeasures,
-            allProjectionListMeasureIndexes);
-    if (measureBlockIndexes.length > 0) {
+    int[] measureChunkIndexes = QueryUtil.getMeasureChunkIndexes(
+        currentBlockQueryMeasures, expressionMeasures,
+        segmentProperties.getMeasuresOrdinalToChunkMapping(), filterMeasures,
+        allProjectionListMeasureIndexes);
+    if (measureChunkIndexes.length > 0) {
 
-      numberOfElementToConsider = measureBlockIndexes[measureBlockIndexes.length - 1]
+      numberOfElementToConsider = measureChunkIndexes[measureChunkIndexes.length - 1]
           == segmentProperties.getMeasures().size() - 1 ?
-          measureBlockIndexes.length - 1 :
-          measureBlockIndexes.length;
+          measureChunkIndexes.length - 1 :
+          measureChunkIndexes.length;
       // setting all the measure chunk indexes to be read from file
-      blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(CarbonUtil
-          .getRangeIndex(measureBlockIndexes, numberOfElementToConsider,
+      blockExecutionInfo.setAllSelectedMeasureIndexRange(
+          CarbonUtil.getRangeIndex(
+              measureChunkIndexes, numberOfElementToConsider,
               numberOfColumnToBeReadInOneIO));
     } else {
-      blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(new int[0][0]);
+      blockExecutionInfo.setAllSelectedMeasureIndexRange(new int[0][0]);
     }
     // setting the indexes of list of dimension in projection list
     blockExecutionInfo.setProjectionListDimensionIndexes(ArrayUtils.toPrimitive(
@@ -413,27 +412,27 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
             .toArray(new Integer[allProjectionListMeasureIndexes.size()])));
     // setting the size of fixed key column (dictionary column)
     blockExecutionInfo
-        .setFixedLengthKeySize(getKeySize(currentBlockQueryDimensions, segmentProperties));
-    Set<Integer> dictionaryColumnBlockIndex = new HashSet<Integer>();
-    List<Integer> noDictionaryColumnBlockIndex = new ArrayList<Integer>();
+        .setFixedLengthKeySize(getKeySize(projectDimensions, segmentProperties));
+    Set<Integer> dictionaryColumnChunkIndex = new HashSet<Integer>();
+    List<Integer> noDictionaryColumnChunkIndex = new ArrayList<Integer>();
     // get the block index to be read from file for query dimension
     // for both dictionary columns and no dictionary columns
-    QueryUtil.fillQueryDimensionsBlockIndexes(currentBlockQueryDimensions,
-        segmentProperties.getDimensionOrdinalToBlockMapping(), dictionaryColumnBlockIndex,
-        noDictionaryColumnBlockIndex);
-    int[] queryDictionaryColumnBlockIndexes = ArrayUtils.toPrimitive(
-        dictionaryColumnBlockIndex.toArray(new Integer[dictionaryColumnBlockIndex.size()]));
+    QueryUtil.fillQueryDimensionChunkIndexes(projectDimensions,
+        segmentProperties.getDimensionOrdinalToChunkMapping(), dictionaryColumnChunkIndex,
+        noDictionaryColumnChunkIndex);
+    int[] queryDictionaryColumnChunkIndexes = ArrayUtils.toPrimitive(
+        dictionaryColumnChunkIndex.toArray(new Integer[dictionaryColumnChunkIndex.size()]));
     // need to sort the dictionary column as for all dimension
     // column key will be filled based on key order
-    Arrays.sort(queryDictionaryColumnBlockIndexes);
-    blockExecutionInfo.setDictionaryColumnBlockIndex(queryDictionaryColumnBlockIndexes);
+    Arrays.sort(queryDictionaryColumnChunkIndexes);
+    blockExecutionInfo.setDictionaryColumnChunkIndex(queryDictionaryColumnChunkIndexes);
     // setting the no dictionary column block indexes
-    blockExecutionInfo.setNoDictionaryBlockIndexes(ArrayUtils.toPrimitive(
-        noDictionaryColumnBlockIndex.toArray(new Integer[noDictionaryColumnBlockIndex.size()])));
+    blockExecutionInfo.setNoDictionaryColumnChunkIndexes(ArrayUtils.toPrimitive(
+        noDictionaryColumnChunkIndex.toArray(new Integer[noDictionaryColumnChunkIndex.size()])));
     // setting each column value size
     blockExecutionInfo.setEachColumnValueSize(segmentProperties.getEachDimColumnValueSize());
     blockExecutionInfo.setComplexColumnParentBlockIndexes(
-        getComplexDimensionParentBlockIndexes(currentBlockQueryDimensions));
+        getComplexDimensionParentBlockIndexes(projectDimensions));
     blockExecutionInfo.setVectorBatchCollector(queryModel.isVectorReader());
     try {
       // to set column group and its key structure info which will be used
@@ -441,15 +440,15 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
       // for getting the column group column data in case of final row
       // and in case of dimension aggregation
       blockExecutionInfo.setColumnGroupToKeyStructureInfo(
-          QueryUtil.getColumnGroupKeyStructureInfo(currentBlockQueryDimensions, segmentProperties));
+          QueryUtil.getColumnGroupKeyStructureInfo(projectDimensions, segmentProperties));
     } catch (KeyGenException e) {
       throw new QueryExecutionException(e);
     }
     // set actual query dimensions and measures. It may differ in case of restructure scenarios
-    blockExecutionInfo.setActualQueryDimensions(queryModel.getQueryDimension()
-        .toArray(new QueryDimension[queryModel.getQueryDimension().size()]));
-    blockExecutionInfo.setActualQueryMeasures(queryModel.getQueryMeasures()
-        .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()]));
+    blockExecutionInfo.setActualQueryDimensions(queryModel.getProjectionDimensions()
+        .toArray(new ProjectionDimension[queryModel.getProjectionDimensions().size()]));
+    blockExecutionInfo.setActualQueryMeasures(queryModel.getProjectionMeasures()
+        .toArray(new ProjectionMeasure[queryModel.getProjectionMeasures().size()]));
     DataTypeUtil.setDataTypeConverter(queryModel.getConverter());
     return blockExecutionInfo;
   }
@@ -464,7 +463,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
    * @param blockMetadataInfo block metadata info
    * @return key size
    */
-  private int getKeySize(List<QueryDimension> queryDimension,
+  private int getKeySize(List<ProjectionDimension> queryDimension,
       SegmentProperties blockMetadataInfo) {
     // add the dimension block ordinal for each dictionary column
     // existing in the current block dimensions. Set is used because in case of column groups
@@ -475,12 +474,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     while (counter < queryDimension.size()) {
       if (queryDimension.get(counter).getDimension().getNumberOfChild() > 0) {
         counter += queryDimension.get(counter).getDimension().getNumberOfChild();
-        continue;
       } else if (!CarbonUtil.hasEncoding(queryDimension.get(counter).getDimension().getEncoder(),
           Encoding.DICTIONARY)) {
         counter++;
       } else {
-        fixedLengthDimensionOrdinal.add(blockMetadataInfo.getDimensionOrdinalToBlockMapping()
+        fixedLengthDimensionOrdinal.add(blockMetadataInfo.getDimensionOrdinalToChunkMapping()
             .get(queryDimension.get(counter).getDimension().getOrdinal()));
         counter++;
       }
@@ -502,26 +500,26 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
   /**
    * Below method will be used to get the measures present in the current block
    *
-   * @param blockExecutionInfo
+   * @param executionInfo
    * @param queryModel         query model
    * @param tableBlock         table block
    * @return
    */
-  private List<QueryMeasure> getCurrentBlockQueryMeasures(BlockExecutionInfo blockExecutionInfo,
+  private List<ProjectionMeasure> getCurrentBlockQueryMeasures(BlockExecutionInfo executionInfo,
       QueryModel queryModel, AbstractIndex tableBlock) throws QueryExecutionException {
     // getting the measure info which will be used while filling up measure data
-    List<QueryMeasure> updatedQueryMeasures = RestructureUtil
-        .createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo,
-            queryModel.getQueryMeasures(), tableBlock.getSegmentProperties().getMeasures());
+    List<ProjectionMeasure> updatedQueryMeasures = RestructureUtil
+        .createMeasureInfoAndGetCurrentBlockQueryMeasures(executionInfo,
+            queryModel.getProjectionMeasures(), tableBlock.getSegmentProperties().getMeasures());
     // setting the measure aggregator for all aggregation function selected
     // in query
-    blockExecutionInfo.getMeasureInfo().setMeasureDataTypes(queryProperties.measureDataTypes);
+    executionInfo.getMeasureInfo().setMeasureDataTypes(queryProperties.measureDataTypes);
     return updatedQueryMeasures;
   }
 
-  private int[] getComplexDimensionParentBlockIndexes(List<QueryDimension> queryDimensions) {
+  private int[] getComplexDimensionParentBlockIndexes(List<ProjectionDimension> queryDimensions) {
     List<Integer> parentBlockIndexList = new ArrayList<Integer>();
-    for (QueryDimension queryDimension : queryDimensions) {
+    for (ProjectionDimension queryDimension : queryDimensions) {
       if (queryDimension.getDimension().getDataType().isComplexType()) {
         parentBlockIndexList.add(queryDimension.getDimension().getOrdinal());
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
index 931b1ab..93d696b 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/DetailQueryExecutor.java
@@ -23,7 +23,7 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.result.iterator.DetailQueryResultIterator;
 
 /**
@@ -31,10 +31,10 @@ import org.apache.carbondata.core.scan.result.iterator.DetailQueryResultIterator
  * For executing the detail query it will pass all the block execution
  * info to detail query result iterator and iterator will be returned
  */
-public class DetailQueryExecutor extends AbstractQueryExecutor<BatchResult> {
+public class DetailQueryExecutor extends AbstractQueryExecutor<RowBatch> {
 
   @Override
-  public CarbonIterator<BatchResult> execute(QueryModel queryModel)
+  public CarbonIterator<RowBatch> execute(QueryModel queryModel)
       throws QueryExecutionException, IOException {
     List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
     this.queryIterator = new DetailQueryResultIterator(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
index 65f2838..06adf8f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/infos/BlockExecutionInfo.java
@@ -21,13 +21,11 @@ import java.util.Map;
 import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.mutate.DeleteDeltaVo;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
 import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 
 /**
  * Below class will have all the properties which needed during query execution
@@ -41,14 +39,6 @@ public class BlockExecutionInfo {
   private AbstractIndex blockIndex;
 
   /**
-   * each segment key size can be different and in that case we need to update
-   * the fixed key with latest segment key generator. so this property will
-   * tell whether this is required or not if key size is same then it is not
-   * required
-   */
-  private boolean isFixedKeyUpdateRequired;
-
-  /**
    * below to store all the information required for measures during query
    * execution
    */
@@ -73,22 +63,22 @@ public class BlockExecutionInfo {
   /**
    * total number of dimension in block
    */
-  private int totalNumberDimensionBlock;
+  private int totalNumberDimensionToRead;
 
   /**
    * total number of measure in block
    */
-  private int totalNumberOfMeasureBlock;
+  private int totalNumberOfMeasureToRead;
 
   /**
    * will be used to read the dimension block from file
    */
-  private int[][] allSelectedDimensionBlocksIndexes;
+  private int[][] allSelectedDimensionColumnIndexRange;
 
   /**
    * will be used to read the measure block from file
    */
-  private int[][] allSelectedMeasureBlocksIndexes;
+  private int[][] allSelectedMeasureIndexRange;
 
   /**
    * list of dimension present in the projection
@@ -118,16 +108,11 @@ public class BlockExecutionInfo {
   /**
    * dictionary column block indexes based on query
    */
-  private int[] dictionaryColumnBlockIndex;
+  private int[] dictionaryColumnChunkIndex;
   /**
    * no dictionary column block indexes in based on the query order
    */
-  private int[] noDictionaryBlockIndexes;
-
-  /**
-   * key generator used for generating the table block fixed length key
-   */
-  private KeyGenerator blockKeyGenerator;
+  private int[] noDictionaryColumnChunkIndexes;
 
   /**
    * each column value size
@@ -180,23 +165,23 @@ public class BlockExecutionInfo {
    * list of dimension present in the current block. This will be
    * different in case of restructured block
    */
-  private QueryDimension[] queryDimensions;
+  private ProjectionDimension[] projectionDimensions;
 
   /**
    * list of dimension selected for in query
    */
-  private QueryDimension[] actualQueryDimensions;
+  private ProjectionDimension[] actualQueryDimensions;
 
   /**
    * list of dimension present in the current block. This will be
    * different in case of restructured block
    */
-  private QueryMeasure[] queryMeasures;
+  private ProjectionMeasure[] projectionMeasures;
 
   /**
    * list of measure selected in query
    */
-  private QueryMeasure[] actualQueryMeasures;
+  private ProjectionMeasure[] actualQueryMeasures;
 
   /**
    * variable to maintain dimension existence and default value info
@@ -214,11 +199,6 @@ public class BlockExecutionInfo {
   private boolean isRestructuredBlock;
 
   /**
-   * absolute table identifier
-   */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-
-  /**
    * delete delta file path
    */
   private String[] deleteDeltaFilePath;
@@ -229,13 +209,6 @@ public class BlockExecutionInfo {
   private boolean prefetchBlocklet = true;
 
   private Map<String, DeleteDeltaVo> deletedRecordsMap;
-  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-  }
 
   /**
    * @param blockIndex the tableBlock to set
@@ -245,20 +218,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the isFixedKeyUpdateRequired
-   */
-  public boolean isFixedKeyUpdateRequired() {
-    return isFixedKeyUpdateRequired;
-  }
-
-  /**
-   * @param isFixedKeyUpdateRequired the isFixedKeyUpdateRequired to set
-   */
-  public void setFixedKeyUpdateRequired(boolean isFixedKeyUpdateRequired) {
-    this.isFixedKeyUpdateRequired = isFixedKeyUpdateRequired;
-  }
-
-  /**
    * @return the aggregatorInfos
    */
   public MeasureInfo getMeasureInfo() {
@@ -301,59 +260,60 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the totalNumberDimensionBlock
+   * @return the totalNumberDimensionToRead
    */
-  public int getTotalNumberDimensionBlock() {
-    return totalNumberDimensionBlock;
+  public int getTotalNumberDimensionToRead() {
+    return totalNumberDimensionToRead;
   }
 
   /**
-   * @param totalNumberDimensionBlock the totalNumberDimensionBlock to set
+   * @param totalNumberDimensionToRead the totalNumberDimensionToRead to set
    */
-  public void setTotalNumberDimensionBlock(int totalNumberDimensionBlock) {
-    this.totalNumberDimensionBlock = totalNumberDimensionBlock;
+  public void setTotalNumberDimensionToRead(int totalNumberDimensionToRead) {
+    this.totalNumberDimensionToRead = totalNumberDimensionToRead;
   }
 
   /**
-   * @return the totalNumberOfMeasureBlock
+   * @return the totalNumberOfMeasureToRead
    */
-  public int getTotalNumberOfMeasureBlock() {
-    return totalNumberOfMeasureBlock;
+  public int getTotalNumberOfMeasureToRead() {
+    return totalNumberOfMeasureToRead;
   }
 
   /**
-   * @param totalNumberOfMeasureBlock the totalNumberOfMeasureBlock to set
+   * @param totalNumberOfMeasureToRead the totalNumberOfMeasureToRead to set
    */
-  public void setTotalNumberOfMeasureBlock(int totalNumberOfMeasureBlock) {
-    this.totalNumberOfMeasureBlock = totalNumberOfMeasureBlock;
+  public void setTotalNumberOfMeasureToRead(int totalNumberOfMeasureToRead) {
+    this.totalNumberOfMeasureToRead = totalNumberOfMeasureToRead;
   }
 
   /**
-   * @return the allSelectedDimensionBlocksIndexes
+   * @return the allSelectedDimensionColumnIndexRange
    */
-  public int[][] getAllSelectedDimensionBlocksIndexes() {
-    return allSelectedDimensionBlocksIndexes;
+  public int[][] getAllSelectedDimensionColumnIndexRange() {
+    return allSelectedDimensionColumnIndexRange;
   }
 
   /**
-   * @param allSelectedDimensionBlocksIndexes the allSelectedDimensionBlocksIndexes to set
+   * @param allSelectedDimensionColumnIndexRange the allSelectedDimensionColumnIndexRange to set
    */
-  public void setAllSelectedDimensionBlocksIndexes(int[][] allSelectedDimensionBlocksIndexes) {
-    this.allSelectedDimensionBlocksIndexes = allSelectedDimensionBlocksIndexes;
+  public void setAllSelectedDimensionColumnIndexRange(int[][] allSelectedDimensionColumnIndexRange)
+  {
+    this.allSelectedDimensionColumnIndexRange = allSelectedDimensionColumnIndexRange;
   }
 
   /**
-   * @return the allSelectedMeasureBlocksIndexes
+   * @return the allSelectedMeasureIndexRange
    */
-  public int[][] getAllSelectedMeasureBlocksIndexes() {
-    return allSelectedMeasureBlocksIndexes;
+  public int[][] getAllSelectedMeasureIndexRange() {
+    return allSelectedMeasureIndexRange;
   }
 
   /**
-   * @param allSelectedMeasureBlocksIndexes the allSelectedMeasureBlocksIndexes to set
+   * @param allSelectedMeasureIndexRange the allSelectedMeasureIndexRange to set
    */
-  public void setAllSelectedMeasureBlocksIndexes(int[][] allSelectedMeasureBlocksIndexes) {
-    this.allSelectedMeasureBlocksIndexes = allSelectedMeasureBlocksIndexes;
+  public void setAllSelectedMeasureIndexRange(int[][] allSelectedMeasureIndexRange) {
+    this.allSelectedMeasureIndexRange = allSelectedMeasureIndexRange;
   }
 
   /**
@@ -413,20 +373,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the tableBlockKeyGenerator
-   */
-  public KeyGenerator getBlockKeyGenerator() {
-    return blockKeyGenerator;
-  }
-
-  /**
-   * @param tableBlockKeyGenerator the tableBlockKeyGenerator to set
-   */
-  public void setBlockKeyGenerator(KeyGenerator tableBlockKeyGenerator) {
-    this.blockKeyGenerator = tableBlockKeyGenerator;
-  }
-
-  /**
    * @return the eachColumnValueSize
    */
   public int[] getEachColumnValueSize() {
@@ -441,31 +387,31 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the dictionaryColumnBlockIndex
+   * @return the dictionaryColumnChunkIndex
    */
-  public int[] getDictionaryColumnBlockIndex() {
-    return dictionaryColumnBlockIndex;
+  public int[] getDictionaryColumnChunkIndex() {
+    return dictionaryColumnChunkIndex;
   }
 
   /**
-   * @param dictionaryColumnBlockIndex the dictionaryColumnBlockIndex to set
+   * @param dictionaryColumnChunkIndex the dictionaryColumnChunkIndex to set
    */
-  public void setDictionaryColumnBlockIndex(int[] dictionaryColumnBlockIndex) {
-    this.dictionaryColumnBlockIndex = dictionaryColumnBlockIndex;
+  public void setDictionaryColumnChunkIndex(int[] dictionaryColumnChunkIndex) {
+    this.dictionaryColumnChunkIndex = dictionaryColumnChunkIndex;
   }
 
   /**
-   * @return the noDictionaryBlockIndexes
+   * @return the noDictionaryColumnChunkIndexes
    */
-  public int[] getNoDictionaryBlockIndexes() {
-    return noDictionaryBlockIndexes;
+  public int[] getNoDictionaryColumnChunkIndexes() {
+    return noDictionaryColumnChunkIndexes;
   }
 
   /**
-   * @param noDictionaryBlockIndexes the noDictionaryBlockIndexes to set
+   * @param noDictionaryColumnChunkIndexes the noDictionaryColumnChunkIndexes to set
    */
-  public void setNoDictionaryBlockIndexes(int[] noDictionaryBlockIndexes) {
-    this.noDictionaryBlockIndexes = noDictionaryBlockIndexes;
+  public void setNoDictionaryColumnChunkIndexes(int[] noDictionaryColumnChunkIndexes) {
+    this.noDictionaryColumnChunkIndexes = noDictionaryColumnChunkIndexes;
   }
 
   /**
@@ -519,20 +465,20 @@ public class BlockExecutionInfo {
     this.complexColumnParentBlockIndexes = complexColumnParentBlockIndexes;
   }
 
-  public QueryDimension[] getQueryDimensions() {
-    return queryDimensions;
+  public ProjectionDimension[] getProjectionDimensions() {
+    return projectionDimensions;
   }
 
-  public void setQueryDimensions(QueryDimension[] queryDimensions) {
-    this.queryDimensions = queryDimensions;
+  public void setProjectionDimensions(ProjectionDimension[] projectionDimensions) {
+    this.projectionDimensions = projectionDimensions;
   }
 
-  public QueryMeasure[] getQueryMeasures() {
-    return queryMeasures;
+  public ProjectionMeasure[] getProjectionMeasures() {
+    return projectionMeasures;
   }
 
-  public void setQueryMeasures(QueryMeasure[] queryMeasures) {
-    this.queryMeasures = queryMeasures;
+  public void setProjectionMeasures(ProjectionMeasure[] projectionMeasures) {
+    this.projectionMeasures = projectionMeasures;
   }
 
   /**
@@ -579,7 +525,8 @@ public class BlockExecutionInfo {
     this.vectorBatchCollector = vectorBatchCollector;
   }
 
-  public String getBlockId() {
+  // Return file name and path, like Part0/Segment_0/part-0-0_batchno0-0-1517155583332.carbondata
+  public String getBlockIdString() {
     return blockId;
   }
 
@@ -603,19 +550,19 @@ public class BlockExecutionInfo {
     this.dimensionInfo = dimensionInfo;
   }
 
-  public QueryDimension[] getActualQueryDimensions() {
+  public ProjectionDimension[] getActualQueryDimensions() {
     return actualQueryDimensions;
   }
 
-  public void setActualQueryDimensions(QueryDimension[] actualQueryDimensions) {
+  public void setActualQueryDimensions(ProjectionDimension[] actualQueryDimensions) {
     this.actualQueryDimensions = actualQueryDimensions;
   }
 
-  public QueryMeasure[] getActualQueryMeasures() {
+  public ProjectionMeasure[] getActualQueryMeasures() {
     return actualQueryMeasures;
   }
 
-  public void setActualQueryMeasures(QueryMeasure[] actualQueryMeasures) {
+  public void setActualQueryMeasures(ProjectionMeasure[] actualQueryMeasures) {
     this.actualQueryMeasures = actualQueryMeasures;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index 8eb0a0d..00cdfe7 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -60,9 +60,8 @@ import org.apache.carbondata.core.scan.filter.GenericQueryType;
 import org.apache.carbondata.core.scan.filter.TableProvider;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -83,7 +82,7 @@ public class QueryUtil {
    * @param keyGenerator    key generator
    * @return masked key
    */
-  public static int[] getMaskedByteRange(List<QueryDimension> queryDimensions,
+  public static int[] getMaskedByteRange(List<ProjectionDimension> queryDimensions,
       KeyGenerator keyGenerator) {
     Set<Integer> byteRangeSet = new TreeSet<Integer>();
     int[] byteRange = null;
@@ -167,7 +166,7 @@ public class QueryUtil {
    * @return max key for dimension
    * @throws KeyGenException if any problem while generating the key
    */
-  public static byte[] getMaxKeyBasedOnDimensions(List<QueryDimension> queryDimensions,
+  public static byte[] getMaxKeyBasedOnDimensions(List<ProjectionDimension> queryDimensions,
       KeyGenerator generator) throws KeyGenException {
     long[] max = new long[generator.getDimCount()];
     Arrays.fill(max, 0L);
@@ -209,46 +208,46 @@ public class QueryUtil {
    * on query dimension
    *
    * @param queryDimensions                query dimension
-   * @param dimensionOrdinalToBlockMapping mapping of dimension block in file to query dimension
+   * @param dimensionOrdinalToChunkMapping 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,
+  public static int[] getDimensionChunkIndexes(List<ProjectionDimension> queryDimensions,
+      Map<Integer, Integer> dimensionOrdinalToChunkMapping,
       List<CarbonDimension> customAggregationDimension, Set<CarbonDimension> filterDimensions,
       Set<Integer> allProjectionListDimensionIndexes) {
     // using set as in row group columns will point to same block
-    Set<Integer> dimensionBlockIndex = new HashSet<Integer>();
+    Set<Integer> dimensionChunkIndex = new HashSet<Integer>();
     Set<Integer> filterDimensionOrdinal = getFilterDimensionOrdinal(filterDimensions);
-    int blockIndex = 0;
+    int chunkIndex = 0;
     for (int i = 0; i < queryDimensions.size(); i++) {
       if (queryDimensions.get(i).getDimension().hasEncoding(Encoding.IMPLICIT)) {
         continue;
       }
 
       Integer dimensionOrdinal = queryDimensions.get(i).getDimension().getOrdinal();
-      allProjectionListDimensionIndexes.add(dimensionOrdinalToBlockMapping.get(dimensionOrdinal));
+      allProjectionListDimensionIndexes.add(dimensionOrdinalToChunkMapping.get(dimensionOrdinal));
       if (queryDimensions.get(i).getDimension().getNumberOfChild() > 0) {
         addChildrenBlockIndex(allProjectionListDimensionIndexes,
             queryDimensions.get(i).getDimension());
       }
 
       if (!filterDimensionOrdinal.contains(dimensionOrdinal)) {
-        blockIndex = dimensionOrdinalToBlockMapping.get(dimensionOrdinal);
-        dimensionBlockIndex.add(blockIndex);
+        chunkIndex = dimensionOrdinalToChunkMapping.get(dimensionOrdinal);
+        dimensionChunkIndex.add(chunkIndex);
         if (queryDimensions.get(i).getDimension().getNumberOfChild() > 0) {
-          addChildrenBlockIndex(dimensionBlockIndex, queryDimensions.get(i).getDimension());
+          addChildrenBlockIndex(dimensionChunkIndex, queryDimensions.get(i).getDimension());
         }
       }
     }
     for (int i = 0; i < customAggregationDimension.size(); i++) {
-      blockIndex =
-          dimensionOrdinalToBlockMapping.get(customAggregationDimension.get(i).getOrdinal());
+      chunkIndex =
+          dimensionOrdinalToChunkMapping.get(customAggregationDimension.get(i).getOrdinal());
       // not adding the children dimension as dimension aggregation
       // is not push down in case of complex dimension
-      dimensionBlockIndex.add(blockIndex);
+      dimensionChunkIndex.add(chunkIndex);
     }
     int[] dimensionIndex = ArrayUtils
-        .toPrimitive(dimensionBlockIndex.toArray(new Integer[dimensionBlockIndex.size()]));
+        .toPrimitive(dimensionChunkIndex.toArray(new Integer[dimensionChunkIndex.size()]));
     Arrays.sort(dimensionIndex);
     return dimensionIndex;
   }
@@ -278,7 +277,7 @@ public class QueryUtil {
    * @throws IOException
    */
   public static Map<String, Dictionary> getDimensionDictionaryDetail(
-      List<QueryDimension> queryDimensions, Set<CarbonDimension> filterComplexDimensions,
+      List<ProjectionDimension> queryDimensions, Set<CarbonDimension> filterComplexDimensions,
       AbsoluteTableIdentifier absoluteTableIdentifier, TableProvider tableProvider)
       throws IOException {
     // to store complex dimension and its child id unique column id list, this is required as
@@ -433,23 +432,23 @@ public class QueryUtil {
    * @param ordinalToBlockIndexMapping measure ordinal to block mapping
    * @return block indexes
    */
-  public static int[] getMeasureBlockIndexes(List<QueryMeasure> queryMeasures,
+  public static int[] getMeasureChunkIndexes(List<ProjectionMeasure> queryMeasures,
       List<CarbonMeasure> expressionMeasure, Map<Integer, Integer> ordinalToBlockIndexMapping,
       Set<CarbonMeasure> filterMeasures, List<Integer> allProjectionListMeasureIdexes) {
-    Set<Integer> measureBlockIndex = new HashSet<Integer>();
+    Set<Integer> measureChunkIndex = new HashSet<Integer>();
     Set<Integer> filterMeasureOrdinal = getFilterMeasureOrdinal(filterMeasures);
     for (int i = 0; i < queryMeasures.size(); i++) {
       Integer measureOrdinal = queryMeasures.get(i).getMeasure().getOrdinal();
       allProjectionListMeasureIdexes.add(measureOrdinal);
       if (!filterMeasureOrdinal.contains(measureOrdinal)) {
-        measureBlockIndex.add(ordinalToBlockIndexMapping.get(measureOrdinal));
+        measureChunkIndex.add(ordinalToBlockIndexMapping.get(measureOrdinal));
       }
     }
     for (int i = 0; i < expressionMeasure.size(); i++) {
-      measureBlockIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal()));
+      measureChunkIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal()));
     }
     int[] measureIndexes =
-        ArrayUtils.toPrimitive(measureBlockIndex.toArray(new Integer[measureBlockIndex.size()]));
+        ArrayUtils.toPrimitive(measureChunkIndex.toArray(new Integer[measureChunkIndex.size()]));
     Arrays.sort(measureIndexes);
     return measureIndexes;
   }
@@ -462,8 +461,8 @@ public class QueryUtil {
    * @param queryDimensions  query dimension
    * @return sort dimension indexes
    */
-  public static byte[] getSortDimensionIndexes(List<QueryDimension> sortedDimensions,
-      List<QueryDimension> queryDimensions) {
+  public static byte[] getSortDimensionIndexes(List<ProjectionDimension> sortedDimensions,
+      List<ProjectionDimension> queryDimensions) {
     byte[] sortedDims = new byte[queryDimensions.size()];
     int indexOf = 0;
     for (int i = 0; i < sortedDims.length; i++) {
@@ -485,7 +484,7 @@ public class QueryUtil {
    * @throws KeyGenException if problem while key generation
    */
   public static Map<Integer, KeyStructureInfo> getColumnGroupKeyStructureInfo(
-      List<QueryDimension> queryDimensions, SegmentProperties segmentProperties)
+      List<ProjectionDimension> queryDimensions, SegmentProperties segmentProperties)
       throws KeyGenException {
     Map<Integer, KeyStructureInfo> rowGroupToItsRSInfo = new HashMap<Integer, KeyStructureInfo>();
     // get column group id and its ordinal mapping of column group
@@ -521,7 +520,7 @@ public class QueryUtil {
       restructureInfos.setMdkeyQueryDimensionOrdinal(ArrayUtils
           .toPrimitive(mdKeyOrdinalForQuery.toArray(new Integer[mdKeyOrdinalForQuery.size()])));
       rowGroupToItsRSInfo
-          .put(segmentProperties.getDimensionOrdinalToBlockMapping().get(ordinal.get(0)),
+          .put(segmentProperties.getDimensionOrdinalToChunkMapping().get(ordinal.get(0)),
               restructureInfos);
     }
     return rowGroupToItsRSInfo;
@@ -552,16 +551,16 @@ public class QueryUtil {
    * stored in bit level
    */
   private static Map<Integer, List<Integer>> getColumnGroupAndItsOrdinalMapping(
-      List<QueryDimension> origdimensions) {
+      List<ProjectionDimension> origDimensions) {
 
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>(origdimensions.size());
-    dimensions.addAll(origdimensions);
+    List<ProjectionDimension> dimensions = new ArrayList<>(origDimensions.size());
+    dimensions.addAll(origDimensions);
     /*
      * sort based on column group id
      */
-    Collections.sort(dimensions, new Comparator<QueryDimension>() {
+    Collections.sort(dimensions, new Comparator<ProjectionDimension>() {
 
-      @Override public int compare(QueryDimension o1, QueryDimension o2) {
+      @Override public int compare(ProjectionDimension o1, ProjectionDimension o2) {
         return Integer
             .compare(o1.getDimension().columnGroupId(), o2.getDimension().columnGroupId());
       }
@@ -636,70 +635,26 @@ public class QueryUtil {
    * 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
+   * @param projectDimensions                  dimension present in the query
+   * @param columnOrdinalToChunkIndexMapping column ordinal to block index mapping
+   * @param dictionaryDimensionChunkIndex    list to store dictionary column block indexes
+   * @param noDictionaryDimensionChunkIndex  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) {
+  public static void fillQueryDimensionChunkIndexes(
+      List<ProjectionDimension> projectDimensions,
+      Map<Integer, Integer> columnOrdinalToChunkIndexMapping,
+      Set<Integer> dictionaryDimensionChunkIndex,
+      List<Integer> noDictionaryDimensionChunkIndex) {
+    for (ProjectionDimension queryDimension : projectDimensions) {
       if (CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)
           && queryDimension.getDimension().getNumberOfChild() == 0) {
-        dictionaryDimensionBlockIndex
-            .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal()));
+        dictionaryDimensionChunkIndex
+            .add(columnOrdinalToChunkIndexMapping.get(queryDimension.getDimension().getOrdinal()));
       } else if (
           !CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.IMPLICIT)
               && queryDimension.getDimension().getNumberOfChild() == 0) {
-        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 query measure
-    for (QueryMeasure queryMeasure : queryModel.getQueryMeasures()) {
-      // in case of count start column name will  be count * so
-      // 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 && !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()));
+        noDictionaryDimensionChunkIndex
+            .add(columnOrdinalToChunkIndexMapping.get(queryDimension.getDimension().getOrdinal()));
       }
     }
   }
@@ -759,11 +714,11 @@ public class QueryUtil {
    * @return complex dimension and query type
    */
   public static Map<Integer, GenericQueryType> getComplexDimensionsMap(
-      List<QueryDimension> queryDimensions, Map<Integer, Integer> dimensionToBlockIndexMap,
+      List<ProjectionDimension> queryDimensions, Map<Integer, Integer> dimensionToBlockIndexMap,
       int[] eachComplexColumnValueSize, Map<String, Dictionary> columnIdToDictionaryMap,
       Set<CarbonDimension> filterDimensions) {
     Map<Integer, GenericQueryType> complexTypeMap = new HashMap<Integer, GenericQueryType>();
-    for (QueryDimension dimension : queryDimensions) {
+    for (ProjectionDimension dimension : queryDimensions) {
       CarbonDimension actualDimension = dimension.getDimension();
       if (actualDimension.getNumberOfChild() == 0) {
         continue;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
index 6500dd7..e1f871a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/RestructureUtil.java
@@ -34,8 +34,8 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.infos.DimensionInfo;
 import org.apache.carbondata.core.scan.executor.infos.MeasureInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
@@ -61,11 +61,11 @@ public class RestructureUtil {
    * @param tableComplexDimension
    * @return list of query dimension which is present in the table block
    */
-  public static List<QueryDimension> createDimensionInfoAndGetCurrentBlockQueryDimension(
-      BlockExecutionInfo blockExecutionInfo, List<QueryDimension> queryDimensions,
+  public static List<ProjectionDimension> createDimensionInfoAndGetCurrentBlockQueryDimension(
+      BlockExecutionInfo blockExecutionInfo, List<ProjectionDimension> queryDimensions,
       List<CarbonDimension> tableBlockDimensions, List<CarbonDimension> tableComplexDimension) {
-    List<QueryDimension> presentDimension =
-        new ArrayList<QueryDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    List<ProjectionDimension> presentDimension =
+        new ArrayList<ProjectionDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     boolean[] isDimensionExists = new boolean[queryDimensions.size()];
     Object[] defaultValues = new Object[queryDimensions.size()];
     // create dimension information instance
@@ -74,22 +74,21 @@ public class RestructureUtil {
     int newNoDictionaryColumnCount = 0;
     // selecting only those dimension which is present in the query
     int dimIndex = 0;
-    for (QueryDimension queryDimension : queryDimensions) {
+    for (ProjectionDimension queryDimension : queryDimensions) {
       if (queryDimension.getDimension().hasEncoding(Encoding.IMPLICIT)) {
         presentDimension.add(queryDimension);
         isDimensionExists[dimIndex] = true;
       } else {
         for (CarbonDimension tableDimension : tableBlockDimensions) {
           if (tableDimension.getColumnId().equals(queryDimension.getDimension().getColumnId())) {
-            QueryDimension currentBlockDimension = new QueryDimension(tableDimension.getColName());
+            ProjectionDimension currentBlockDimension = new ProjectionDimension(tableDimension);
             tableDimension.getColumnSchema()
                 .setPrecision(queryDimension.getDimension().getColumnSchema().getPrecision());
             tableDimension.getColumnSchema()
                 .setScale(queryDimension.getDimension().getColumnSchema().getScale());
             tableDimension.getColumnSchema()
                 .setDefaultValue(queryDimension.getDimension().getDefaultValue());
-            currentBlockDimension.setDimension(tableDimension);
-            currentBlockDimension.setQueryOrder(queryDimension.getQueryOrder());
+            currentBlockDimension.setOrdinal(queryDimension.getOrdinal());
             presentDimension.add(currentBlockDimension);
             isDimensionExists[dimIndex] = true;
             break;
@@ -102,11 +101,10 @@ public class RestructureUtil {
         }
         for (CarbonDimension tableDimension : tableComplexDimension) {
           if (tableDimension.getColumnId().equals(queryDimension.getDimension().getColumnId())) {
-            QueryDimension currentBlockDimension = new QueryDimension(tableDimension.getColName());
+            ProjectionDimension currentBlockDimension = new ProjectionDimension(tableDimension);
             // TODO: for complex dimension set scale and precision by traversing
             // the child dimensions
-            currentBlockDimension.setDimension(tableDimension);
-            currentBlockDimension.setQueryOrder(queryDimension.getQueryOrder());
+            currentBlockDimension.setOrdinal(queryDimension.getOrdinal());
             presentDimension.add(currentBlockDimension);
             isDimensionExists[dimIndex] = true;
             break;
@@ -242,39 +240,6 @@ public class RestructureUtil {
   }
 
   /**
-   * Below method is to add dimension children for complex type dimension as
-   * internally we are creating dimension column for each each complex
-   * dimension so when complex query dimension request will come in the query,
-   * we need to add its children as it is hidden from the user For example if
-   * complex dimension is of Array of String[2] so we are storing 3 dimension
-   * and when user will query for complex type i.e. array type we need to add
-   * its children and then we will read respective block and create a tuple
-   * based on all three dimension
-   *
-   * @param queryDimensions      current query dimensions
-   * @param tableBlockDimensions dimensions which is present in the table block
-   * @return updated dimension(after adding complex type children)
-   */
-  public static List<CarbonDimension> addChildrenForComplexTypeDimension(
-      List<CarbonDimension> queryDimensions, List<CarbonDimension> tableBlockDimensions) {
-    List<CarbonDimension> updatedQueryDimension = new ArrayList<CarbonDimension>();
-    int numberOfChildren = 0;
-    for (CarbonDimension queryDimension : queryDimensions) {
-      // if number of child is zero, then it is not a complex dimension
-      // so directly add it query dimension
-      if (queryDimension.getNumberOfChild() == 0) {
-        updatedQueryDimension.add(queryDimension);
-      }
-      // if number of child is more than 1 then add all its children
-      numberOfChildren = queryDimension.getOrdinal() + queryDimension.getNumberOfChild();
-      for (int j = queryDimension.getOrdinal(); j < numberOfChildren; j++) {
-        updatedQueryDimension.add(tableBlockDimensions.get(j));
-      }
-    }
-    return updatedQueryDimension;
-  }
-
-  /**
    * Method for computing measure default value based on the data type
    *
    * @param columnSchema
@@ -367,30 +332,29 @@ public class RestructureUtil {
    * @param currentBlockMeasures current block measures
    * @return measures present in the block
    */
-  public static List<QueryMeasure> createMeasureInfoAndGetCurrentBlockQueryMeasures(
-      BlockExecutionInfo blockExecutionInfo, List<QueryMeasure> queryMeasures,
+  public static List<ProjectionMeasure> createMeasureInfoAndGetCurrentBlockQueryMeasures(
+      BlockExecutionInfo blockExecutionInfo, List<ProjectionMeasure> queryMeasures,
       List<CarbonMeasure> currentBlockMeasures) {
     MeasureInfo measureInfo = new MeasureInfo();
-    List<QueryMeasure> presentMeasure = new ArrayList<>(queryMeasures.size());
+    List<ProjectionMeasure> presentMeasure = new ArrayList<>(queryMeasures.size());
     int numberOfMeasureInQuery = queryMeasures.size();
     List<Integer> measureOrdinalList = new ArrayList<>(numberOfMeasureInQuery);
     Object[] defaultValues = new Object[numberOfMeasureInQuery];
     boolean[] measureExistsInCurrentBlock = new boolean[numberOfMeasureInQuery];
     int index = 0;
-    for (QueryMeasure queryMeasure : queryMeasures) {
+    for (ProjectionMeasure queryMeasure : queryMeasures) {
       // if query measure exists in current dimension measures
       // then setting measure exists is true
       // otherwise adding a default value of a measure
       for (CarbonMeasure carbonMeasure : currentBlockMeasures) {
         if (carbonMeasure.getColumnId().equals(queryMeasure.getMeasure().getColumnId())) {
-          QueryMeasure currentBlockMeasure = new QueryMeasure(carbonMeasure.getColName());
+          ProjectionMeasure currentBlockMeasure = new ProjectionMeasure(carbonMeasure);
           carbonMeasure.getColumnSchema().setDataType(queryMeasure.getMeasure().getDataType());
           carbonMeasure.getColumnSchema().setPrecision(queryMeasure.getMeasure().getPrecision());
           carbonMeasure.getColumnSchema().setScale(queryMeasure.getMeasure().getScale());
           carbonMeasure.getColumnSchema()
               .setDefaultValue(queryMeasure.getMeasure().getDefaultValue());
-          currentBlockMeasure.setMeasure(carbonMeasure);
-          currentBlockMeasure.setQueryOrder(queryMeasure.getQueryOrder());
+          currentBlockMeasure.setOrdinal(queryMeasure.getOrdinal());
           presentMeasure.add(currentBlockMeasure);
           measureOrdinalList.add(carbonMeasure.getOrdinal());
           measureExistsInCurrentBlock[index] = true;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java
index 981efb5..5cd2d34 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/ColumnExpression.java
@@ -70,10 +70,6 @@ public class ColumnExpression extends LeafExpression {
     return columnName;
   }
 
-  public void setColumnName(String columnName) {
-    this.columnName = columnName;
-  }
-
   public boolean isDimension() {
     return isDimension;
   }
@@ -102,10 +98,6 @@ public class ColumnExpression extends LeafExpression {
     return dataType;
   }
 
-  public void setDataType(DataType dataType) {
-    this.dataType = dataType;
-  }
-
   @Override public ExpressionResult evaluate(RowIntf value) {
     return new ExpressionResult(dataType, (null == value ? null : value.getVal(colIndex)));
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java
index dad3975..8b060e1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/FilterModificationNode.java
@@ -17,62 +17,29 @@
 
 package org.apache.carbondata.core.scan.expression;
 
-import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
 
 public class FilterModificationNode {
   private Expression currentExp;
   private Expression parentExp;
   private ExpressionType expType;
-  private ExpressionResult literalValue;
-  private String columnName;
 
-  public FilterModificationNode(Expression currentNode, Expression parentNode,
-      ExpressionType expType, DataType dataType, Object literalVal, String colName) {
+  FilterModificationNode(Expression currentNode, Expression parentNode, ExpressionType expType) {
     this.currentExp = currentNode;
     this.parentExp = parentNode;
     this.expType = expType;
-    this.columnName = colName;
-    this.literalValue = new ExpressionResult(dataType, literalVal);
   }
 
   public Expression getCurrentExp() {
     return currentExp;
   }
 
-  public void setCurrentExp(Expression currentExp) {
-    this.currentExp = currentExp;
-  }
-
   public Expression getParentExp() {
     return parentExp;
   }
 
-  public void setParentExp(Expression parentExp) {
-    this.parentExp = parentExp;
-  }
-
   public ExpressionType getExpType() {
     return expType;
   }
 
-  public void setExpType(ExpressionType expType) {
-    this.expType = expType;
-  }
-
-  public ExpressionResult getLiteralValue() {
-    return literalValue;
-  }
-
-  public void setLiteralValue(ExpressionResult literalValue) {
-    this.literalValue = literalValue;
-  }
-
-  public String getColumnName() {
-    return columnName;
-  }
-
-  public void setColumnName(String columnName) {
-    this.columnName = columnName;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
index e7cc1c3..585bf60 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/expression/RangeExpressionEvaluator.java
@@ -48,7 +48,6 @@ public class RangeExpressionEvaluator {
       LogServiceFactory.getLogService(RangeExpressionEvaluator.class.getName());
   private Expression expr;
   private Expression srcNode;
-  private Expression srcParentNode;
   private Expression tarNode;
   private Expression tarParentNode;
 
@@ -64,15 +63,15 @@ public class RangeExpressionEvaluator {
     this.expr = expr;
   }
 
-  public Expression getSrcNode() {
+  private Expression getSrcNode() {
     return srcNode;
   }
 
-  public void setTarNode(Expression expr) {
+  private void setTarNode(Expression expr) {
     this.tarNode = expr;
   }
 
-  public void setTarParentNode(Expression expr) {
+  private void setTarParentNode(Expression expr) {
     this.tarParentNode = expr;
   }
 
@@ -237,12 +236,10 @@ public class RangeExpressionEvaluator {
   private void addFilterExpressionMap(Map<String, List<FilterModificationNode>> filterExpressionMap,
       Expression currentNode, Expression parentNode) {
     String colName = getColumnName(currentNode);
-    DataType dataType = getLiteralDataType(currentNode);
-    Object literalVal = getLiteralValue(currentNode);
     ExpressionType expType = getExpressionType(currentNode);
 
     FilterModificationNode filterExpression =
-        new FilterModificationNode(currentNode, parentNode, expType, dataType, literalVal, colName);
+        new FilterModificationNode(currentNode, parentNode, expType);
 
     if (null == filterExpressionMap.get(colName)) {
       filterExpressionMap.put(colName, new ArrayList<FilterModificationNode>());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index 3e23aa3..b882b51 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -105,8 +105,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
    *
    */
   public List<DataRefNode> getFilterredBlocks(DataRefNode btreeNode,
-      FilterResolverIntf filterResolver, AbstractIndex tableSegment,
-      AbsoluteTableIdentifier tableIdentifier) {
+      FilterResolverIntf filterResolver, AbstractIndex tableSegment) {
     // 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
@@ -401,7 +400,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
       case FALSE:
         return new FalseConditionalResolverImpl(expression, false, false, tableIdentifier);
       case TRUE:
-        return new TrueConditionalResolverImpl(expression, false, false, tableIdentifier);
+        return new TrueConditionalResolverImpl(expression, false, false);
       case EQUALS:
         currentCondExpression = (BinaryConditionalExpression) expression;
         // check for implicit column in the expression
@@ -410,7 +409,6 @@ public class FilterExpressionProcessor implements FilterProcessor {
               currentCondExpression.getColumnList().get(0).getCarbonColumn();
           if (carbonColumn.hasEncoding(Encoding.IMPLICIT)) {
             return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-                tableIdentifier,
                 currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
           }
         }
@@ -436,7 +434,6 @@ public class FilterExpressionProcessor implements FilterProcessor {
                   tableIdentifier);
             }
             return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-                tableIdentifier,
                 currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
           }
           // getting new dim index.
@@ -462,14 +459,12 @@ public class FilterExpressionProcessor implements FilterProcessor {
             }
           }
           return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-              tableIdentifier,
               currentCondExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
 
         }
         break;
       case RANGE:
-        return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true,
-            tableIdentifier, false);
+        return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true, false);
       case NOT_EQUALS:
         currentCondExpression = (BinaryConditionalExpression) expression;
         column = currentCondExpression.getColumnList().get(0).getCarbonColumn();
@@ -492,8 +487,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
               return new RowLevelRangeFilterResolverImpl(expression, isExpressionResolve, false,
                   tableIdentifier);
             }
-            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false,
-                tableIdentifier, true);
+            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, true);
           }
 
           if (!currentCondExpression.getColumnList().get(0).getCarbonColumn()
@@ -516,11 +510,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
                   tableIdentifier);
             }
 
-            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false,
-                tableIdentifier, false);
+            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, false);
           }
-          return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false,
-              tableIdentifier, false);
+          return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false, false);
         }
         break;
 
@@ -534,7 +526,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
                 .hasEncoding(Encoding.DICTIONARY) && !condExpression.getColumnList().get(0)
                 .getCarbonColumn().hasEncoding(Encoding.DIRECT_DICTIONARY))
                 || (condExpression.getColumnList().get(0).getCarbonColumn().isMeasure())) {
-              return new ConditionalFilterResolverImpl(expression, true, true, tableIdentifier,
+              return new ConditionalFilterResolverImpl(expression, true, true,
                   condExpression.getColumnList().get(0).getCarbonColumn().isMeasure());
             }
           }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
index dc392ba..dbb6eb7 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterProcessor.java
@@ -53,7 +53,7 @@ public interface FilterProcessor {
    * @return list of DataRefNode.
    */
   List<DataRefNode> getFilterredBlocks(DataRefNode dataRefNode, FilterResolverIntf filterResolver,
-      AbstractIndex segmentIndexBuilder, AbsoluteTableIdentifier tableIdentifier);
+      AbstractIndex segmentIndexBuilder);
 
   /**
    * This API will get the map of required partitions.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
index 8dcac30..0b72e77 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterUtil.java
@@ -49,7 +49,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -172,10 +172,8 @@ public final class FilterUtil {
                   segmentProperties);
         case RANGE:
           return new RangeValueFilterExecuterImpl(
-              ((ConditionalFilterResolverImpl) filterExpressionResolverTree)
-                  .getDimColResolvedFilterInfo(),
-              null, filterExpressionResolverTree.getFilterExpression(),
-              ((ConditionalFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
+              filterExpressionResolverTree.getDimColResolvedFilterInfo(),
+              filterExpressionResolverTree.getFilterExpression(),
               ((ConditionalFilterResolverImpl) filterExpressionResolverTree)
                   .getFilterRangeValues(segmentProperties), segmentProperties);
         case TRUE:
@@ -551,10 +549,9 @@ public final class FilterUtil {
       sortFilterModelMembers(columnExpression, evaluateResultList);
       getDictionaryValue(evaluateResultList, forwardDictionary, surrogates);
       filterInfo =
-          getFilterValues(columnExpression, forwardDictionary, isIncludeFilter, null, surrogates);
+          getFilterValues(forwardDictionary, isIncludeFilter, surrogates);
       if (filterInfo.isOptimized()) {
-        return getDimColumnFilterInfoAfterApplyingCBO(columnExpression,
-            forwardDictionary, filterInfo);
+        return getDimColumnFilterInfoAfterApplyingCBO(forwardDictionary, filterInfo);
       }
     } finally {
       CarbonUtil.clearDictionaryCache(forwardDictionary);
@@ -566,22 +563,20 @@ public final class FilterUtil {
    * Method will prepare the  dimfilterinfo instance by resolving the filter
    * expression value to its respective surrogates.
    *
-   * @param columnExpression
    * @param forwardDictionary
    * @param isIncludeFilter
    * @param filterInfo
    * @param surrogates
    * @return
    */
-  private static ColumnFilterInfo getFilterValues(ColumnExpression columnExpression,
-      Dictionary forwardDictionary, boolean isIncludeFilter, ColumnFilterInfo filterInfo,
-      List<Integer> surrogates) throws QueryExecutionException {
+  private static ColumnFilterInfo getFilterValues(Dictionary forwardDictionary,
+      boolean isIncludeFilter, List<Integer> surrogates) {
     // Default value has to be added
     if (surrogates.isEmpty()) {
       surrogates.add(0);
     }
     boolean isExcludeFilterNeedsToApply = false;
-    if (null == filterInfo && isIncludeFilter) {
+    if (isIncludeFilter) {
       isExcludeFilterNeedsToApply =
           isExcludeFilterNeedsToApply(forwardDictionary, surrogates.size());
     }
@@ -593,17 +588,10 @@ public final class FilterUtil {
         columnFilterInfo.setOptimized(true);
       }
       columnFilterInfo.setIncludeFilter(isIncludeFilter);
-      if (null != filterInfo) {
-        filterInfo.setIncludeFilter(isIncludeFilter);
-        filterInfo.setOptimized(true);
-        filterInfo.setExcludeFilterList(surrogates);
-        return filterInfo;
+      if (!isIncludeFilter) {
+        columnFilterInfo.setExcludeFilterList(surrogates);
       } else {
-        if (!isIncludeFilter) {
-          columnFilterInfo.setExcludeFilterList(surrogates);
-        } else {
-          columnFilterInfo.setFilterList(surrogates);
-        }
+        columnFilterInfo.setFilterList(surrogates);
       }
     }
     return columnFilterInfo;
@@ -619,8 +607,7 @@ public final class FilterUtil {
   }
 
   private static ColumnFilterInfo getDimColumnFilterInfoAfterApplyingCBO(
-      ColumnExpression columnExpression, Dictionary forwardDictionary,
-      ColumnFilterInfo filterInfo) throws FilterUnsupportedException, QueryExecutionException {
+      Dictionary forwardDictionary, ColumnFilterInfo filterInfo) throws FilterUnsupportedException {
     List<Integer> excludeMemberSurrogates =
         prepareExcludeFilterMembers(forwardDictionary, filterInfo.getFilterList());
     filterInfo.setExcludeFilterList(excludeMemberSurrogates);
@@ -730,14 +717,12 @@ public final class FilterUtil {
       prepareIncludeFilterMembers(expression, columnExpression, isIncludeFilter, forwardDictionary,
           surrogates);
       ColumnFilterInfo filterInfo =
-          getFilterValues(columnExpression, forwardDictionary, isIncludeFilter, null, surrogates);
+          getFilterValues(forwardDictionary, isIncludeFilter, surrogates);
       if (filterInfo.isOptimized()) {
-        return getDimColumnFilterInfoAfterApplyingCBO(columnExpression, forwardDictionary,
+        return getDimColumnFilterInfoAfterApplyingCBO(forwardDictionary,
             filterInfo);
       }
       return filterInfo;
-    } catch (QueryExecutionException e) {
-      throw new FilterUnsupportedException(e.getMessage());
     } finally {
       CarbonUtil.clearDictionaryCache(forwardDictionary);
     }
@@ -761,21 +746,16 @@ public final class FilterUtil {
    * expression value to its respective surrogates in the scenario of restructure.
    *
    * @param expression
-   * @param columnExpression
    * @param defaultValues
    * @param defaultSurrogate
    * @return
    * @throws FilterUnsupportedException
    */
-  public static ColumnFilterInfo getFilterListForRS(Expression expression,
-      ColumnExpression columnExpression, String defaultValues, int defaultSurrogate)
-      throws FilterUnsupportedException {
+  public static ColumnFilterInfo getFilterListForRS(Expression expression, String defaultValues,
+      int defaultSurrogate) throws FilterUnsupportedException {
     List<Integer> filterValuesList = new ArrayList<Integer>(20);
     ColumnFilterInfo 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) {
@@ -821,8 +801,6 @@ public final class FilterUtil {
     List<String> evaluateResultListFinal = new ArrayList<String>(20);
     ColumnFilterInfo columnFilterInfo = null;
 
-    // KeyGenerator keyGenerator =
-    // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate });
     try {
       RowIntf row = new RowImpl();
       if (defaultValues.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
@@ -1775,14 +1753,14 @@ public final class FilterUtil {
    * This method will compare the selected data against null values and
    * flip the bitSet if any null value is found
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param bitSet
    */
-  public static void removeNullValues(DimensionColumnDataChunk dimensionColumnDataChunk,
+  public static void removeNullValues(DimensionColumnPage dimensionColumnPage,
       BitSet bitSet, byte[] defaultValue) {
     if (!bitSet.isEmpty()) {
       for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-        if (dimensionColumnDataChunk.compareTo(i, defaultValue) == 0) {
+        if (dimensionColumnPage.compareTo(i, defaultValue) == 0) {
           bitSet.flip(i);
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java
index 3742e7e..214bd9d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/GenericQueryType.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 
 import org.apache.spark.sql.types.DataType;
 
@@ -45,7 +45,7 @@ public interface GenericQueryType {
 
   DataType getSchemaType();
 
-  void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder) throws IOException;
+  void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder) throws IOException;
 
   Object getDataBasedOnDataTypeFromSurrogates(ByteBuffer surrogateData);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
index f882162..e309451 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/AndFilterExecuterImpl.java
@@ -21,7 +21,7 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilterExecutor {
@@ -35,18 +35,18 @@ public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilt
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    BitSetGroup leftFilters = leftExecuter.applyFilter(blockChunkHolder, useBitsetPipeLine);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    BitSetGroup leftFilters = leftExecuter.applyFilter(rawBlockletColumnChunks, useBitsetPipeLine);
     if (leftFilters.isEmpty()) {
       return leftFilters;
     }
-    BitSetGroup rightFilter = rightExecuter.applyFilter(blockChunkHolder, useBitsetPipeLine);
+    BitSetGroup rightFilter = rightExecuter.applyFilter(rawBlockletColumnChunks, useBitsetPipeLine);
     if (rightFilter.isEmpty()) {
       return rightFilter;
     }
     leftFilters.and(rightFilter);
-    blockChunkHolder.setBitSetGroup(leftFilters);
+    rawBlockletColumnChunks.setBitSetGroup(leftFilters);
     return leftFilters;
   }
 
@@ -69,9 +69,10 @@ public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilt
     return leftFilters;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    leftExecuter.readBlocks(blocksChunkHolder);
-    rightExecuter.readBlocks(blocksChunkHolder);
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    leftExecuter.readColumnChunks(rawBlockletColumnChunks);
+    rightExecuter.readColumnChunks(rawBlockletColumnChunks);
   }
 
   @Override
@@ -93,8 +94,7 @@ public class AndFilterExecuterImpl implements FilterExecuter, ImplicitColumnFilt
       rightFilter = ((ImplicitColumnFilterExecutor) rightExecuter)
           .isFilterValuesPresentInBlockOrBlocklet(maxValue, minValue, uniqueBlockPath);
     } else {
-      rightFilter = rightExecuter
-          .isScanRequired(maxValue, minValue);
+      rightFilter = rightExecuter.isScanRequired(maxValue, minValue);
     }
     if (rightFilter.isEmpty()) {
       return rightFilter;


[06/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
index f51ced3..6a401d8 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionExecutor.java
@@ -34,20 +34,16 @@ import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
 
 /**
  * Executor class for executing the query on the selected segments to be merged.
@@ -70,6 +66,9 @@ public class CarbonCompactionExecutor {
    */
   private boolean restructuredBlockExists;
 
+  // converter for UTF8String and decimal conversion
+  private DataTypeConverter dataTypeConverter;
+
   /**
    * Constructor
    *
@@ -82,13 +81,14 @@ public class CarbonCompactionExecutor {
   public CarbonCompactionExecutor(Map<String, TaskBlockInfo> segmentMapping,
       SegmentProperties segmentProperties, CarbonTable carbonTable,
       Map<String, List<DataFileFooter>> dataFileMetadataSegMapping,
-      boolean restructuredBlockExists) {
+      boolean restructuredBlockExists, DataTypeConverter dataTypeConverter) {
     this.segmentMapping = segmentMapping;
     this.destinationSegProperties = segmentProperties;
     this.carbonTable = carbonTable;
     this.dataFileMetadataSegMapping = dataFileMetadataSegMapping;
     this.restructuredBlockExists = restructuredBlockExists;
-    queryExecutorList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.queryExecutorList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.dataTypeConverter = dataTypeConverter;
   }
 
   /**
@@ -100,7 +100,9 @@ public class CarbonCompactionExecutor {
     List<RawResultIterator> resultList =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     List<TableBlockInfo> list = null;
-    queryModel = prepareQueryModel(list);
+    queryModel = carbonTable.createQueryModelWithProjectAllColumns(dataTypeConverter);
+    queryModel.setReadPageByPage(enablePageLevelReaderForCompaction());
+    queryModel.setForcedDetailRawQuery(true);
     // iterate each seg ID
     for (Map.Entry<String, TaskBlockInfo> taskMap : segmentMapping.entrySet()) {
       String segmentId = taskMap.getKey();
@@ -156,7 +158,7 @@ public class CarbonCompactionExecutor {
    * @param blockList
    * @return
    */
-  private CarbonIterator<BatchResult> executeBlockList(List<TableBlockInfo> blockList)
+  private CarbonIterator<RowBatch> executeBlockList(List<TableBlockInfo> blockList)
       throws QueryExecutionException, IOException {
     queryModel.setTableBlockInfos(blockList);
     QueryExecutor queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
@@ -195,48 +197,6 @@ public class CarbonCompactionExecutor {
   }
 
   /**
-   * Preparing of the query model.
-   *
-   * @param blockList
-   * @return
-   */
-  private QueryModel prepareQueryModel(List<TableBlockInfo> blockList) {
-    QueryModel model = new QueryModel();
-    model.setTableBlockInfos(blockList);
-    model.setForcedDetailRawQuery(true);
-    model.setFilterExpressionResolverTree(null);
-    model.setConverter(DataTypeUtil.getDataTypeConverter());
-    model.setReadPageByPage(enablePageLevelReaderForCompaction());
-
-    List<QueryDimension> dims = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    List<CarbonDimension> dimensions =
-        carbonTable.getDimensionByTableName(carbonTable.getTableName());
-    for (CarbonDimension dim : dimensions) {
-      // check if dimension is deleted
-      QueryDimension queryDimension = new QueryDimension(dim.getColName());
-      queryDimension.setDimension(dim);
-      dims.add(queryDimension);
-    }
-    model.setQueryDimension(dims);
-
-    List<QueryMeasure> msrs = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<CarbonMeasure> measures =
-        carbonTable.getMeasureByTableName(carbonTable.getTableName());
-    for (CarbonMeasure carbonMeasure : measures) {
-      // check if measure is deleted
-      QueryMeasure queryMeasure = new QueryMeasure(carbonMeasure.getColName());
-      queryMeasure.setMeasure(carbonMeasure);
-      msrs.add(queryMeasure);
-    }
-    model.setQueryMeasures(msrs);
-    model.setQueryId(System.nanoTime() + "");
-    model.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
-    model.setTable(carbonTable);
-    return model;
-  }
-
-  /**
    * Whether to enable page level reader for compaction or not.
    */
   private boolean enablePageLevelReaderForCompaction() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java
index 79e9e5a..b6f12a5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.java
@@ -23,7 +23,6 @@ import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
 import org.apache.carbondata.processing.partition.DataPartitioner;
 import org.apache.carbondata.processing.partition.Partition;
 
@@ -46,9 +45,8 @@ public final class QueryPartitionHelper {
   /**
    * Get partitions applicable for query based on filters applied in query
    */
-  public List<Partition> getPartitionsForQuery(CarbonQueryPlan queryPlan) {
-    String tableUniqueName =
-        CarbonTable.buildUniqueName(queryPlan.getDatabaseName(), queryPlan.getTableName());
+  public List<Partition> getPartitionsForQuery(String databaseName, String tableName) {
+    String tableUniqueName = CarbonTable.buildUniqueName(databaseName, tableName);
 
     DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
index 36e022b..01db4f6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.java
@@ -18,7 +18,6 @@
 package org.apache.carbondata.processing.partition.spliter;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -26,19 +25,14 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 public abstract class AbstractCarbonQueryExecutor {
@@ -47,8 +41,8 @@ public abstract class AbstractCarbonQueryExecutor {
       LogServiceFactory.getLogService(AbstractCarbonQueryExecutor.class.getName());
   protected CarbonTable carbonTable;
   protected QueryModel queryModel;
-  protected QueryExecutor queryExecutor;
-  protected Map<String, TaskBlockInfo> segmentMapping;
+  private QueryExecutor queryExecutor;
+  Map<String, TaskBlockInfo> segmentMapping;
 
   /**
    * get executor and execute the query model.
@@ -56,7 +50,7 @@ public abstract class AbstractCarbonQueryExecutor {
    * @param blockList
    * @return
    */
-  protected CarbonIterator<BatchResult> executeBlockList(List<TableBlockInfo> blockList)
+  CarbonIterator<RowBatch> executeBlockList(List<TableBlockInfo> blockList)
       throws QueryExecutionException, IOException {
     queryModel.setTableBlockInfos(blockList);
     this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
@@ -64,46 +58,6 @@ public abstract class AbstractCarbonQueryExecutor {
   }
 
   /**
-   * Preparing of the query model.
-   *
-   * @param blockList
-   * @return
-   */
-  protected QueryModel prepareQueryModel(List<TableBlockInfo> blockList) {
-    QueryModel model = new QueryModel();
-    model.setTableBlockInfos(blockList);
-    model.setForcedDetailRawQuery(true);
-    model.setFilterExpressionResolverTree(null);
-
-    List<QueryDimension> dims = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    List<CarbonDimension> dimensions =
-        carbonTable.getDimensionByTableName(carbonTable.getTableName());
-    for (CarbonDimension dim : dimensions) {
-      // check if dimension is deleted
-      QueryDimension queryDimension = new QueryDimension(dim.getColName());
-      queryDimension.setDimension(dim);
-      dims.add(queryDimension);
-    }
-    model.setQueryDimension(dims);
-
-    List<QueryMeasure> msrs = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<CarbonMeasure> measures =
-        carbonTable.getMeasureByTableName(carbonTable.getTableName());
-    for (CarbonMeasure carbonMeasure : measures) {
-      // check if measure is deleted
-      QueryMeasure queryMeasure = new QueryMeasure(carbonMeasure.getColName());
-      queryMeasure.setMeasure(carbonMeasure);
-      msrs.add(queryMeasure);
-    }
-    model.setQueryMeasures(msrs);
-    model.setQueryId(System.nanoTime() + "");
-    model.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
-    model.setTable(carbonTable);
-    return model;
-  }
-
-  /**
    * Below method will be used
    * for cleanup
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
index 6afec0b..b18207d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
@@ -31,6 +31,7 @@ import org.apache.carbondata.core.datastore.block.TaskBlockInfo;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
 import org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResultIterator;
+import org.apache.carbondata.core.util.DataTypeConverterImpl;
 
 /**
  * Used to read carbon blocks when add/split partition
@@ -48,7 +49,8 @@ public class CarbonSplitExecutor extends AbstractCarbonQueryExecutor {
   public List<PartitionSpliterRawResultIterator> processDataBlocks(String segmentId)
       throws QueryExecutionException, IOException {
     List<TableBlockInfo> list = null;
-    queryModel = prepareQueryModel(list);
+    queryModel = carbonTable.createQueryModelWithProjectAllColumns(new DataTypeConverterImpl());
+    queryModel.setForcedDetailRawQuery(true);
     List<PartitionSpliterRawResultIterator> resultList
         = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     TaskBlockInfo taskBlockInfo = segmentMapping.get(segmentId);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
index ec91472..4abdf3c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
+import org.apache.carbondata.core.scan.model.QueryProjection;
 import org.apache.carbondata.processing.partition.Partition;
 import org.apache.carbondata.processing.partition.impl.DefaultLoadBalancer;
 import org.apache.carbondata.processing.partition.impl.PartitionMultiFileImpl;
@@ -46,7 +46,7 @@ public class CarbonQueryUtil {
    * It creates the one split for each region server.
    */
   public static synchronized TableSplit[] getTableSplits(String databaseName, String tableName,
-      CarbonQueryPlan queryPlan) {
+      QueryProjection queryPlan) {
 
     //Just create splits depends on locations of region servers
     List<Partition> allPartitions = null;
@@ -55,7 +55,7 @@ public class CarbonQueryUtil {
           QueryPartitionHelper.getInstance().getAllPartitions(databaseName, tableName);
     } else {
       allPartitions =
-          QueryPartitionHelper.getInstance().getPartitionsForQuery(queryPlan);
+          QueryPartitionHelper.getInstance().getPartitionsForQuery(databaseName, tableName);
     }
     TableSplit[] splits = new TableSplit[allPartitions.size()];
     for (int i = 0; i < splits.length; i++) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
index 897a812..790f9d8 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
@@ -155,7 +155,7 @@ class StreamHandoffRDD[K, V](
     CarbonTableInputFormat.setTableInfo(hadoopConf, carbonTable.getTableInfo)
     val attemptContext = new TaskAttemptContextImpl(hadoopConf, attemptId)
     val format = new CarbonTableInputFormat[Array[Object]]()
-    val model = format.getQueryModel(inputSplit, attemptContext)
+    val model = format.createQueryModel(inputSplit, attemptContext)
     val inputFormat = new CarbonStreamInputFormat
     val streamReader = inputFormat.createRecordReader(inputSplit, attemptContext)
       .asInstanceOf[CarbonStreamRecordReader]


[33/49] carbondata git commit: Revert "[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading"

Posted by ja...@apache.org.
Revert "[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading"

This reverts commit 6dd8b038fc898dbf48ad30adfc870c19eb38e3d0.


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

Branch: refs/heads/carbonstore-rebase4
Commit: 7a11f8e5feda416ee7f14dd2118bff6826789482
Parents: a6bf77f
Author: Jacky Li <ja...@qq.com>
Authored: Sat Feb 10 10:34:59 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:05:07 2018 +0800

----------------------------------------------------------------------
 .../constants/CarbonLoadOptionConstants.java    |  10 -
 .../core/datastore/block/TableBlockInfo.java    |  29 --
 .../carbondata/core/util/CarbonProperties.java  |  11 -
 docs/useful-tips-on-carbondata.md               |   1 -
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |   4 +-
 .../spark/sql/hive/DistributionUtil.scala       |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  18 +-
 .../merger/NodeMultiBlockRelation.java          |  40 --
 .../processing/util/CarbonLoaderUtil.java       | 494 +++++++------------
 .../processing/util/CarbonLoaderUtilTest.java   | 125 -----
 10 files changed, 183 insertions(+), 551 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index a6bf60f..bcfeba0 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,14 +114,4 @@ public final class CarbonLoadOptionConstants {
    */
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 10000000;
 
-  /**
-   * enable block size based block allocation while loading data. By default, carbondata assigns
-   * blocks to node based on block number. If this option is set to `true`, carbondata will
-   * consider block size first and make sure that all the nodes will process almost equal size of
-   * data. This option is especially useful when you encounter skewed data.
-   */
-  @CarbonProperty
-  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
-      = "carbon.load.skewedDataOptimization.enabled";
-  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index c0cebe0..a7bfdba 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,8 +19,6 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
-import java.util.Arrays;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -100,20 +98,6 @@ public class TableBlockInfo implements Distributable, Serializable {
 
   private String dataMapWriterPath;
 
-  /**
-   * comparator to sort by block size in descending order.
-   * Since each line is not exactly the same, the size of a InputSplit may differs,
-   * so we allow some deviation for these splits.
-   */
-  public static final Comparator<Distributable> DATA_SIZE_DESC_COMPARATOR =
-      new Comparator<Distributable>() {
-        @Override public int compare(Distributable o1, Distributable o2) {
-          long diff =
-              ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) o2).getBlockLength();
-          return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
-        }
-      };
-
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
       String[] locations, long blockLength, ColumnarFormatVersion version,
       String[] deletedDeltaFilePath) {
@@ -450,17 +434,4 @@ public class TableBlockInfo implements Distributable, Serializable {
   public void setDataMapWriterPath(String dataMapWriterPath) {
     this.dataMapWriterPath = dataMapWriterPath;
   }
-
-  @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder("TableBlockInfo{");
-    sb.append("filePath='").append(filePath).append('\'');
-    sb.append(", blockOffset=").append(blockOffset);
-    sb.append(", blockLength=").append(blockLength);
-    sb.append(", segmentId='").append(segmentId).append('\'');
-    sb.append(", blockletId='").append(blockletId).append('\'');
-    sb.append(", locations=").append(Arrays.toString(locations));
-    sb.append('}');
-    return sb.toString();
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index 474988c..9d52669 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -1235,17 +1235,6 @@ public final class CarbonProperties {
       return CarbonCommonConstants.CARBON_SORT_TEMP_COMPRESSOR_DEFAULT;
     }
   }
-
-  /**
-   * whether optimization for skewed data is enabled
-   * @return true, if enabled; false for not enabled.
-   */
-  public boolean isLoadSkewedDataOptimizationEnabled() {
-    String skewedEnabled = getProperty(
-        CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION,
-        CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT);
-    return skewedEnabled.equalsIgnoreCase("true");
-  }
   /**
    * returns true if carbon property
    * @param key

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/docs/useful-tips-on-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/useful-tips-on-carbondata.md b/docs/useful-tips-on-carbondata.md
index ff339d0..4d43003 100644
--- a/docs/useful-tips-on-carbondata.md
+++ b/docs/useful-tips-on-carbondata.md
@@ -169,6 +169,5 @@
   | carbon.use.local.dir | spark/carbonlib/carbon.properties | Data loading | Whether use YARN local directories for multi-table load disk load balance | If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance. |
   | carbon.use.multiple.temp.dir | spark/carbonlib/carbon.properties | Data loading | Whether to use multiple YARN local directories during table data loading for disk load balance | After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading. |
   | carbon.sort.temp.compressor | spark/carbonlib/carbon.properties | Data loading | Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading. | The optional values are 'SNAPPY','GZIP','BZIP2','LZ4' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck. |
-  | carbon.load.skewedDataOptimization.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable size based block allocation strategy for data loading. | When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB. |
 
   Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 97e3061..8fc80b7 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -281,10 +281,8 @@ class NewCarbonDataLoadRDD[K, V](
         val format = new CSVInputFormat
 
         val split = theSplit.asInstanceOf[CarbonNodePartition]
-        val inputSize = split.blocksDetails.map(_.getBlockLength).sum * 0.1 * 10  / 1024 / 1024
         logInfo("Input split: " + split.serializableHadoopSplit)
-        logInfo("The block count in this node: " + split.nodeBlocksDetail.length)
-        logInfo(f"The input data size in this node: $inputSize%.2fMB")
+        logInfo("The Block Count in this node :" + split.nodeBlocksDetail.length)
         CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordHostBlockMap(
             split.serializableHadoopSplit, split.nodeBlocksDetail.length)
         carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
index a676dd8..1958d61 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
@@ -128,7 +128,7 @@ object DistributionUtil {
    */
   def ensureExecutorsAndGetNodeList(blockList: Seq[Distributable],
       sparkContext: SparkContext): Seq[String] = {
-    val nodeMapping = CarbonLoaderUtil.nodeBlockMapping(blockList.asJava)
+    val nodeMapping = CarbonLoaderUtil.getRequiredExecutors(blockList.asJava)
     ensureExecutorsByNumberAndGetNodeList(nodeMapping, blockList, sparkContext)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index acde951..8d3110a 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -1026,16 +1026,10 @@ object CarbonDataRDDFactory {
     val startTime = System.currentTimeMillis
     val activeNodes = DistributionUtil
       .ensureExecutorsAndGetNodeList(blockList, sqlContext.sparkContext)
-    val skewedDataOptimization = CarbonProperties.getInstance()
-      .isLoadSkewedDataOptimizationEnabled()
-    val blockAssignStrategy = if (skewedDataOptimization) {
-      CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_SIZE_FIRST
-    } else {
-      CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_NUM_FIRST
-    }
-    LOGGER.info(s"Allocating block to nodes using strategy: $blockAssignStrategy")
-    val nodeBlockMapping = CarbonLoaderUtil.nodeBlockMapping(blockList.toSeq.asJava, -1,
-      activeNodes.toList.asJava, blockAssignStrategy).asScala.toSeq
+    val nodeBlockMapping =
+      CarbonLoaderUtil
+        .nodeBlockMapping(blockList.toSeq.asJava, -1, activeNodes.toList.asJava).asScala
+        .toSeq
     val timeElapsed: Long = System.currentTimeMillis - startTime
     LOGGER.info("Total Time taken in block allocation: " + timeElapsed)
     LOGGER.info(s"Total no of blocks: ${ blockList.length }, " +
@@ -1043,9 +1037,7 @@ object CarbonDataRDDFactory {
     var str = ""
     nodeBlockMapping.foreach { entry =>
       val tableBlock = entry._2
-      val totalSize = tableBlock.asScala.map(_.asInstanceOf[TableBlockInfo].getBlockLength).sum
-      str = str + "#Node: " + entry._1 + ", no.of.blocks: " + tableBlock.size() +
-            f", totalsize.of.blocks: ${totalSize * 0.1 * 10 / 1024 /1024}%.2fMB"
+      str = str + "#Node: " + entry._1 + " no.of.blocks: " + tableBlock.size()
       tableBlock.asScala.foreach(tableBlockInfo =>
         if (!tableBlockInfo.getLocations.exists(hostentry =>
           hostentry.equalsIgnoreCase(entry._1)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java b/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
index 1bb5736..ec2ddaf 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
@@ -16,41 +16,15 @@
  */
 package org.apache.carbondata.processing.merger;
 
-import java.util.Comparator;
 import java.util.List;
 
 import org.apache.carbondata.core.datastore.block.Distributable;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 
 public class NodeMultiBlockRelation implements Comparable<NodeMultiBlockRelation> {
 
   private final List<Distributable> blocks;
   private final String node;
 
-  /**
-   * comparator to sort by data size in descending order. This is used to assign big blocks to
-   * bigger nodes first.
-   */
-  public static final Comparator<NodeMultiBlockRelation> DATA_SIZE_DESC_COMPARATOR =
-      new Comparator<NodeMultiBlockRelation>() {
-        @Override
-        public int compare(NodeMultiBlockRelation o1, NodeMultiBlockRelation o2) {
-          long diff = o1.getTotalSizeOfBlocks() - o2.getTotalSizeOfBlocks();
-          return diff > 0 ? -1 : (diff < 0 ? 1 : 0);
-        }
-      };
-  /**
-   * comparator to sort by data size in ascending order. This is used to assign left over blocks to
-   * smaller nodes first.
-   */
-  public static final Comparator<NodeMultiBlockRelation> DATA_SIZE_ASC_COMPARATOR =
-      new Comparator<NodeMultiBlockRelation>() {
-        @Override
-        public int compare(NodeMultiBlockRelation o1, NodeMultiBlockRelation o2) {
-          long diff = o1.getTotalSizeOfBlocks() - o2.getTotalSizeOfBlocks();
-          return diff > 0 ? 1 : (diff < 0 ? -1 : 0);
-        }
-      };
   public NodeMultiBlockRelation(String node, List<Distributable> blocks) {
     this.node = node;
     this.blocks = blocks;
@@ -65,20 +39,6 @@ public class NodeMultiBlockRelation implements Comparable<NodeMultiBlockRelation
     return node;
   }
 
-  /**
-   * get the total size of the blocks
-   * @return size in bytes
-   */
-  public long getTotalSizeOfBlocks() {
-    long totalSize = 0;
-    if (blocks.get(0) instanceof TableBlockInfo) {
-      for (Distributable block : blocks) {
-        totalSize += ((TableBlockInfo) block).getBlockLength();
-      }
-    }
-    return totalSize;
-  }
-
   @Override public int compareTo(NodeMultiBlockRelation obj) {
     return this.blocks.size() - obj.getBlocks().size();
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index a3e889a..e9bd3b8 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -24,16 +24,7 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.charset.Charset;
 import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -45,7 +36,6 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.Distributable;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -66,6 +56,7 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.merger.NodeBlockRelation;
 import org.apache.carbondata.processing.merger.NodeMultiBlockRelation;
 import static org.apache.carbondata.core.enums.EscapeSequences.*;
 
@@ -80,23 +71,6 @@ public final class CarbonLoaderUtil {
   private CarbonLoaderUtil() {
   }
 
-  /**
-   * strategy for assign blocks to nodes/executors
-   */
-  public enum BlockAssignmentStrategy {
-    BLOCK_NUM_FIRST("Assign blocks to node base on number of blocks"),
-    BLOCK_SIZE_FIRST("Assign blocks to node base on data size of blocks");
-    private String name;
-    BlockAssignmentStrategy(String name) {
-      this.name = name;
-    }
-
-    @Override
-    public String toString() {
-      return this.getClass().getSimpleName() + ':' + this.name;
-    }
-  }
-
   public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
     String segmentPath = CarbonTablePath.getSegmentPath(
         loadModel.getTablePath(), currentLoad + "");
@@ -518,9 +492,9 @@ public final class CarbonLoaderUtil {
   public static Map<String, List<List<Distributable>>> nodeBlockTaskMapping(
       List<Distributable> blockInfos, int noOfNodesInput, int parallelism,
       List<String> activeNode) {
+
     Map<String, List<Distributable>> mapOfNodes =
-        CarbonLoaderUtil.nodeBlockMapping(blockInfos, noOfNodesInput, activeNode,
-            BlockAssignmentStrategy.BLOCK_NUM_FIRST);
+        CarbonLoaderUtil.nodeBlockMapping(blockInfos, noOfNodesInput, activeNode);
     int taskPerNode = parallelism / mapOfNodes.size();
     //assigning non zero value to noOfTasksPerNode
     int noOfTasksPerNode = taskPerNode == 0 ? 1 : taskPerNode;
@@ -536,8 +510,7 @@ public final class CarbonLoaderUtil {
    */
   public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos,
       int noOfNodesInput) {
-    return nodeBlockMapping(blockInfos, noOfNodesInput, null,
-        BlockAssignmentStrategy.BLOCK_NUM_FIRST);
+    return nodeBlockMapping(blockInfos, noOfNodesInput, null);
   }
 
   /**
@@ -552,59 +525,82 @@ public final class CarbonLoaderUtil {
   }
 
   /**
+   * the method returns the number of required executors
+   *
+   * @param blockInfos
+   * @return
+   */
+  public static Map<String, List<Distributable>> getRequiredExecutors(
+      List<Distributable> blockInfos) {
+    List<NodeBlockRelation> flattenedList =
+        new ArrayList<NodeBlockRelation>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    for (Distributable blockInfo : blockInfos) {
+      try {
+        for (String eachNode : blockInfo.getLocations()) {
+          NodeBlockRelation nbr = new NodeBlockRelation(blockInfo, eachNode);
+          flattenedList.add(nbr);
+        }
+      } catch (IOException e) {
+        throw new RuntimeException("error getting location of block: " + blockInfo.toString(), e);
+      }
+    }
+    // sort the flattened data.
+    Collections.sort(flattenedList);
+    Map<String, List<Distributable>> nodeAndBlockMapping =
+        new LinkedHashMap<String, List<Distributable>>(
+            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    // from the flattened list create a mapping of node vs Data blocks.
+    createNodeVsBlockMapping(flattenedList, nodeAndBlockMapping);
+    return nodeAndBlockMapping;
+  }
+
+  /**
    * This method will divide the blocks among the nodes as per the data locality
    *
-   * @param blockInfos blocks
+   * @param blockInfos
    * @param noOfNodesInput -1 if number of nodes has to be decided
    *                       based on block location information
-   * @param blockAssignmentStrategy strategy used to assign blocks
-   * @return a map that maps node to blocks
+   * @return
    */
-  public static Map<String, List<Distributable>> nodeBlockMapping(
-      List<Distributable> blockInfos, int noOfNodesInput, List<String> activeNodes,
-      BlockAssignmentStrategy blockAssignmentStrategy) {
-    ArrayList<NodeMultiBlockRelation> rtnNode2Blocks = new ArrayList<>();
-
-    Set<Distributable> uniqueBlocks = new HashSet<>(blockInfos);
-    ArrayList<NodeMultiBlockRelation> originNode2Blocks = createNode2BlocksMapping(blockInfos);
-    Set<String> nodes = new HashSet<>(originNode2Blocks.size());
-    for (NodeMultiBlockRelation relation : originNode2Blocks) {
-      nodes.add(relation.getNode());
-    }
+  public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos,
+      int noOfNodesInput, List<String> activeNodes) {
+
+    Map<String, List<Distributable>> nodeBlocksMap =
+        new HashMap<String, List<Distributable>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+    List<NodeBlockRelation> flattenedList =
+        new ArrayList<NodeBlockRelation>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+    Set<Distributable> uniqueBlocks =
+        new HashSet<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    Set<String> nodes = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+    createFlattenedListFromMap(blockInfos, flattenedList, uniqueBlocks, nodes);
 
     int noofNodes = (-1 == noOfNodesInput) ? nodes.size() : noOfNodesInput;
     if (null != activeNodes) {
       noofNodes = activeNodes.size();
     }
+    int blocksPerNode = blockInfos.size() / noofNodes;
+    blocksPerNode = blocksPerNode <= 0 ? 1 : blocksPerNode;
 
-    // calculate the average expected size for each node
-    long sizePerNode = 0;
-    if (BlockAssignmentStrategy.BLOCK_NUM_FIRST == blockAssignmentStrategy) {
-      sizePerNode = blockInfos.size() / noofNodes;
-      sizePerNode = sizePerNode <= 0 ? 1 : sizePerNode;
-    } else if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
-      long totalFileSize = 0;
-      for (Distributable blockInfo : uniqueBlocks) {
-        totalFileSize += ((TableBlockInfo) blockInfo).getBlockLength();
-      }
-      sizePerNode = totalFileSize / noofNodes;
-    }
+    // sort the flattened data.
+    Collections.sort(flattenedList);
+
+    Map<String, List<Distributable>> nodeAndBlockMapping =
+        new LinkedHashMap<String, List<Distributable>>(
+            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+    // from the flattened list create a mapping of node vs Data blocks.
+    createNodeVsBlockMapping(flattenedList, nodeAndBlockMapping);
 
-    // assign blocks to each node
-    assignBlocksByDataLocality(rtnNode2Blocks, sizePerNode, uniqueBlocks, originNode2Blocks,
-        activeNodes, blockAssignmentStrategy);
+    // so now we have a map of node vs blocks. allocate the block as per the order
+    createOutputMap(nodeBlocksMap, blocksPerNode, uniqueBlocks, nodeAndBlockMapping, activeNodes);
 
     // if any blocks remain then assign them to nodes in round robin.
-    assignLeftOverBlocks(rtnNode2Blocks, uniqueBlocks, sizePerNode, activeNodes,
-        blockAssignmentStrategy);
+    assignLeftOverBlocks(nodeBlocksMap, uniqueBlocks, blocksPerNode, activeNodes);
 
-    // convert
-    Map<String, List<Distributable>> rtnNodeBlocksMap =
-        new HashMap<String, List<Distributable>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (NodeMultiBlockRelation relation : rtnNode2Blocks) {
-      rtnNodeBlocksMap.put(relation.getNode(), relation.getBlocks());
-    }
-    return rtnNodeBlocksMap;
+    return nodeBlocksMap;
   }
 
   /**
@@ -679,207 +675,92 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * If any left over data blocks are present then assign those to nodes in round robin way. This
-   * will not obey the data locality.
+   * If any left over data blocks are present then assign those to nodes in round robin way.
+   *
+   * @param outputMap
+   * @param uniqueBlocks
    */
-  private static void assignLeftOverBlocks(ArrayList<NodeMultiBlockRelation> outputMap,
-      Set<Distributable> leftOverBlocks, long expectedSizePerNode, List<String> activeNodes,
-      BlockAssignmentStrategy blockAssignmentStrategy) {
-    Map<String, Integer> node2Idx = new HashMap<>(outputMap.size());
-    for (int idx = 0; idx < outputMap.size(); idx++) {
-      node2Idx.put(outputMap.get(idx).getNode(), idx);
-    }
+  private static void assignLeftOverBlocks(Map<String, List<Distributable>> outputMap,
+      Set<Distributable> uniqueBlocks, int noOfBlocksPerNode, List<String> activeNodes) {
 
-    // iterate all the nodes and try to allocate blocks to the nodes
     if (activeNodes != null) {
       for (String activeNode : activeNodes) {
-        if (LOGGER.isDebugEnabled()) {
-          LOGGER.debug("Second assignment iteration: assign for executor: " + activeNode);
-        }
-
-        Integer idx;
-        List<Distributable> blockLst;
-        if (node2Idx.containsKey(activeNode)) {
-          idx = node2Idx.get(activeNode);
-          blockLst = outputMap.get(idx).getBlocks();
-        } else {
-          idx = node2Idx.size();
+        List<Distributable> blockLst = outputMap.get(activeNode);
+        if (null == blockLst) {
           blockLst = new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
         }
-        populateBlocks(leftOverBlocks, expectedSizePerNode, blockLst, blockAssignmentStrategy);
-
-        if (!node2Idx.containsKey(activeNode) && blockLst.size() > 0) {
-          outputMap.add(idx, new NodeMultiBlockRelation(activeNode, blockLst));
-          node2Idx.put(activeNode, idx);
+        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
+        if (blockLst.size() > 0) {
+          outputMap.put(activeNode, blockLst);
         }
       }
     } else {
-      for (NodeMultiBlockRelation entry : outputMap) {
-        List<Distributable> blockLst = entry.getBlocks();
-        populateBlocks(leftOverBlocks, expectedSizePerNode, blockLst, blockAssignmentStrategy);
+      for (Map.Entry<String, List<Distributable>> entry : outputMap.entrySet()) {
+        List<Distributable> blockLst = entry.getValue();
+        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
       }
-    }
-
-    // if there is still blocks left, allocate them in round robin manner to each nodes
-    assignBlocksUseRoundRobin(outputMap, leftOverBlocks, blockAssignmentStrategy);
-  }
 
-  /**
-   * assign remaining blocks to nodes
-   *
-   * @param remainingBlocks blocks to be allocated
-   * @param expectedSizePerNode expected size for each node
-   * @param blockLst destination for the blocks to be allocated
-   * @param blockAssignmentStrategy block assignment stretegy
-   */
-  private static void populateBlocks(Set<Distributable> remainingBlocks,
-      long expectedSizePerNode, List<Distributable> blockLst,
-      BlockAssignmentStrategy blockAssignmentStrategy) {
-    switch (blockAssignmentStrategy) {
-      case BLOCK_NUM_FIRST:
-        populateBlocksByNum(remainingBlocks, expectedSizePerNode, blockLst);
-        break;
-      case BLOCK_SIZE_FIRST:
-        populateBlocksBySize(remainingBlocks, expectedSizePerNode, blockLst);
-        break;
-      default:
-        throw new IllegalArgumentException(
-            "Unsupported block assignment strategy: " + blockAssignmentStrategy);
     }
-  }
 
-  /**
-   * Taken N number of distributable blocks from {@param remainingBlocks} and add them to output
-   * {@param blockLst}. After added, the total number of {@param blockLst} is less
-   * than {@param expectedSizePerNode}.
-   */
-  private static void populateBlocksByNum(Set<Distributable> remainingBlocks,
-      long expectedSizePerNode, List<Distributable> blockLst) {
-    Iterator<Distributable> blocks = remainingBlocks.iterator();
-    // if the node is already having the per block nodes then avoid assign the extra blocks
-    if (blockLst.size() == expectedSizePerNode) {
-      return;
-    }
-    while (blocks.hasNext()) {
-      Distributable block = blocks.next();
-      blockLst.add(block);
-      blocks.remove();
-      if (blockLst.size() >= expectedSizePerNode) {
-        break;
+    for (Map.Entry<String, List<Distributable>> entry : outputMap.entrySet()) {
+      Iterator<Distributable> blocks = uniqueBlocks.iterator();
+      if (blocks.hasNext()) {
+        Distributable block = blocks.next();
+        List<Distributable> blockLst = entry.getValue();
+        blockLst.add(block);
+        blocks.remove();
       }
     }
   }
 
   /**
-   * Taken N number of distributable blocks from {@param remainingBlocks} and add them to output
-   * {@param blockLst}. After added, the total accumulated block size of {@param blockLst}
-   * is less than {@param expectedSizePerNode}.
+   * The method populate the blockLst to be allocate to a specific node.
+   * @param uniqueBlocks
+   * @param noOfBlocksPerNode
+   * @param blockLst
    */
-  private static void populateBlocksBySize(Set<Distributable> remainingBlocks,
-      long expectedSizePerNode, List<Distributable> blockLst) {
-    Iterator<Distributable> blocks = remainingBlocks.iterator();
-    //if the node is already having the avg node size then avoid assign the extra blocks
-    long fileSize = 0;
-    for (Distributable block : blockLst) {
-      fileSize += ((TableBlockInfo) block).getBlockLength();
-    }
-    if (fileSize >= expectedSizePerNode) {
-      LOGGER.debug("Capacity is full, skip allocate blocks on this node");
+  private static void populateBlocks(Set<Distributable> uniqueBlocks, int noOfBlocksPerNode,
+      List<Distributable> blockLst) {
+    Iterator<Distributable> blocks = uniqueBlocks.iterator();
+    //if the node is already having the per block nodes then avoid assign the extra blocks
+    if (blockLst.size() == noOfBlocksPerNode) {
       return;
     }
-
     while (blocks.hasNext()) {
       Distributable block = blocks.next();
-      long thisBlockSize = ((TableBlockInfo) block).getBlockLength();
-      if (fileSize < expectedSizePerNode) {
-        // `fileSize==0` means there are no blocks assigned to this node before
-        if (fileSize == 0 || fileSize + thisBlockSize <= expectedSizePerNode * 1.1D) {
-          blockLst.add(block);
-          if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Second Assignment iteration: "
-                + ((TableBlockInfo) block).getFilePath() + "-"
-                + ((TableBlockInfo) block).getBlockLength() + "-->currentNode");
-          }
-          fileSize += thisBlockSize;
-          blocks.remove();
-        }
-      } else {
+      blockLst.add(block);
+      blocks.remove();
+      if (blockLst.size() >= noOfBlocksPerNode) {
         break;
       }
     }
   }
 
   /**
-   * allocate the blocks in round robin manner
+   * To create the final output of the Node and Data blocks
+   *
+   * @param outputMap
+   * @param blocksPerNode
+   * @param uniqueBlocks
+   * @param nodeAndBlockMapping
+   * @param activeNodes
    */
-  private static void assignBlocksUseRoundRobin(ArrayList<NodeMultiBlockRelation> node2Blocks,
-      Set<Distributable> remainingBlocks, BlockAssignmentStrategy blockAssignmentStrategy) {
-    switch (blockAssignmentStrategy) {
-      case BLOCK_NUM_FIRST:
-        roundRobinAssignBlocksByNum(node2Blocks, remainingBlocks);
-        break;
-      case BLOCK_SIZE_FIRST:
-        roundRobinAssignBlocksBySize(node2Blocks, remainingBlocks);
-        break;
-      default:
-        throw new IllegalArgumentException("Unsupported block assignment strategy: "
-            + blockAssignmentStrategy);
-    }
-  }
+  private static void createOutputMap(Map<String, List<Distributable>> outputMap, int blocksPerNode,
+      Set<Distributable> uniqueBlocks, Map<String, List<Distributable>> nodeAndBlockMapping,
+      List<String> activeNodes) {
 
-  private static void roundRobinAssignBlocksByNum(ArrayList<NodeMultiBlockRelation> outputMap,
-      Set<Distributable> remainingBlocks) {
-    for (NodeMultiBlockRelation relation: outputMap) {
-      Iterator<Distributable> blocks = remainingBlocks.iterator();
-      if (blocks.hasNext()) {
-        Distributable block = blocks.next();
-        List<Distributable> blockLst = relation.getBlocks();
-        blockLst.add(block);
-        blocks.remove();
-      }
+    ArrayList<NodeMultiBlockRelation> multiBlockRelations =
+        new ArrayList<>(nodeAndBlockMapping.size());
+    for (Map.Entry<String, List<Distributable>> entry : nodeAndBlockMapping.entrySet()) {
+      multiBlockRelations.add(new NodeMultiBlockRelation(entry.getKey(), entry.getValue()));
     }
-  }
+    // sort nodes based on number of blocks per node, so that nodes having lesser blocks
+    // are assigned first
+    Collections.sort(multiBlockRelations);
 
-  private static void roundRobinAssignBlocksBySize(ArrayList<NodeMultiBlockRelation> outputMap,
-      Set<Distributable> remainingBlocks) {
-    Iterator<Distributable> blocks = remainingBlocks.iterator();
-    while (blocks.hasNext()) {
-      // sort the allocated node-2-blocks in ascending order, the total data size of first one is
-      // the smallest, so we assign this block to it.
-      Collections.sort(outputMap, NodeMultiBlockRelation.DATA_SIZE_ASC_COMPARATOR);
-      Distributable block = blocks.next();
-      List<Distributable> blockLst = outputMap.get(0).getBlocks();
-      blockLst.add(block);
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("RoundRobin assignment iteration: "
-            + ((TableBlockInfo) block).getFilePath() + "-"
-            + ((TableBlockInfo) block).getBlockLength() + "-->" + outputMap.get(0).getNode());
-      }
-      blocks.remove();
-    }
-  }
-  /**
-   * allocate distributable blocks to nodes based on data locality
-   */
-  private static void assignBlocksByDataLocality(
-      ArrayList<NodeMultiBlockRelation> outputNode2Blocks,
-      long expectedSizePerNode, Set<Distributable> remainingBlocks,
-      List<NodeMultiBlockRelation> inputNode2Blocks, List<String> activeNodes,
-      BlockAssignmentStrategy blockAssignmentStrategy) {
-    if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
-      // sort nodes based on data size of all blocks per node, so that nodes having bigger size
-      // are assigned first
-      Collections.sort(inputNode2Blocks, NodeMultiBlockRelation.DATA_SIZE_DESC_COMPARATOR);
-    } else {
-      // sort nodes based on number of blocks per node, so that nodes having lesser blocks
-      // are assigned first
-      Collections.sort(inputNode2Blocks);
-    }
-
-    Map<String, Integer> executor2Idx = new HashMap<>();
-    for (NodeMultiBlockRelation nodeMultiBlockRelation : inputNode2Blocks) {
+    for (NodeMultiBlockRelation nodeMultiBlockRelation : multiBlockRelations) {
       String nodeName = nodeMultiBlockRelation.getNode();
-      // assign the block to the node only if the node is active
+      //assign the block to the node only if the node is active
       String activeExecutor = nodeName;
       if (null != activeNodes) {
         activeExecutor = getActiveExecutor(activeNodes, nodeName);
@@ -887,75 +768,29 @@ public final class CarbonLoaderUtil {
           continue;
         }
       }
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("First Assignment iteration: assign for executor: " + activeExecutor);
-      }
-
-      List<Distributable> blocksInThisNode = nodeMultiBlockRelation.getBlocks();
-      if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
-        // sort blocks based on block size, so that bigger blocks will be assigned first
-        Collections.sort(blocksInThisNode, TableBlockInfo.DATA_SIZE_DESC_COMPARATOR);
-      }
-
-      long nodeCapacity = 0;
+      // this loop will be for each NODE
+      int nodeCapacity = 0;
       // loop thru blocks of each Node
       for (Distributable block : nodeMultiBlockRelation.getBlocks()) {
-        if (!remainingBlocks.contains(block)) {
-          // this block has been added before
-          continue;
-        }
-        // this is the first time to add block to this node, initialize it
-        if (!executor2Idx.containsKey(activeExecutor)) {
-          Integer idx = executor2Idx.size();
-          outputNode2Blocks.add(idx, new NodeMultiBlockRelation(activeExecutor,
-              new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE)));
-          executor2Idx.put(activeExecutor, idx);
-        }
 
-        // assign this block to this node if node has capacity left
-        if (BlockAssignmentStrategy.BLOCK_NUM_FIRST == blockAssignmentStrategy) {
-          if (nodeCapacity < expectedSizePerNode) {
-            Integer idx = executor2Idx.get(activeExecutor);
-            List<Distributable> infos = outputNode2Blocks.get(idx).getBlocks();
+        // check if this is already assigned.
+        if (uniqueBlocks.contains(block)) {
+
+          if (null == outputMap.get(activeExecutor)) {
+            List<Distributable> list =
+                new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+            outputMap.put(activeExecutor, list);
+          }
+          // assign this block to this node if node has capacity left
+          if (nodeCapacity < blocksPerNode) {
+            List<Distributable> infos = outputMap.get(activeExecutor);
             infos.add(block);
             nodeCapacity++;
-            if (LOGGER.isDebugEnabled()) {
-              LOGGER.debug(
-                  "First Assignment iteration: " + ((TableBlockInfo) block).getFilePath() + '-'
-                      + ((TableBlockInfo) block).getBlockLength() + "-->" + activeExecutor);
-            }
-            remainingBlocks.remove(block);
-          } else {
-            // No need to continue loop as node is full
-            break;
-          }
-        } else if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
-          long thisBlockSize = ((TableBlockInfo) block).getBlockLength();
-          // `nodeCapacity == 0` means that there is a huge block that already exceed the
-          // `expectedSize` of the node, so we have to assign it to some node, otherwise it will
-          // be assigned in the last RoundRobin iteration.
-          if (nodeCapacity == 0 || nodeCapacity < expectedSizePerNode) {
-            if (nodeCapacity == 0 || nodeCapacity + thisBlockSize <= expectedSizePerNode * 1.05D) {
-              Integer idx = executor2Idx.get(activeExecutor);
-              List<Distributable> blocks = outputNode2Blocks.get(idx).getBlocks();
-              blocks.add(block);
-              nodeCapacity += thisBlockSize;
-              if (LOGGER.isDebugEnabled()) {
-                LOGGER.debug(
-                    "First Assignment iteration: " + ((TableBlockInfo) block).getFilePath() + '-'
-                        + ((TableBlockInfo) block).getBlockLength() + "-->" + activeExecutor);
-              }
-              remainingBlocks.remove(block);
-            }
-            // this block is too big for current node and there are still capacity left
-            // for small files, so continue to allocate block on this node in next iteration.
+            uniqueBlocks.remove(block);
           } else {
             // No need to continue loop as node is full
             break;
           }
-        } else {
-          throw new IllegalArgumentException(
-              "Unsupported block assignment strategy: " + blockAssignmentStrategy);
         }
       }
     }
@@ -999,37 +834,60 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * Create node to blocks mapping
+   * Create the Node and its related blocks Mapping and put in a Map
    *
-   * @param blockInfos input block info
+   * @param flattenedList
+   * @param nodeAndBlockMapping
    */
-  private static ArrayList<NodeMultiBlockRelation> createNode2BlocksMapping(
-      List<Distributable> blockInfos) {
-    Map<String, Integer> node2Idx = new HashMap<>();
-    ArrayList<NodeMultiBlockRelation> node2Blocks = new ArrayList<>();
+  private static void createNodeVsBlockMapping(List<NodeBlockRelation> flattenedList,
+      Map<String, List<Distributable>> nodeAndBlockMapping) {
+    for (NodeBlockRelation nbr : flattenedList) {
+      String node = nbr.getNode();
+      List<Distributable> list;
+
+      if (null == nodeAndBlockMapping.get(node)) {
+        list = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        list.add(nbr.getBlock());
+        nodeAndBlockMapping.put(node, list);
+      } else {
+        list = nodeAndBlockMapping.get(node);
+        list.add(nbr.getBlock());
+      }
+    }
+    /*for resolving performance issue, removed values() with entrySet () iterating the values and
+    sorting it.entrySet will give the logical view for hashMap and we dont query the map twice for
+    each key whereas values () iterate twice*/
+    Iterator<Map.Entry<String, List<Distributable>>> iterator =
+        nodeAndBlockMapping.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Collections.sort(iterator.next().getValue());
+    }
+  }
 
+  /**
+   * Create the flat List i.e flattening of the Map.
+   *
+   * @param blockInfos
+   * @param flattenedList
+   * @param uniqueBlocks
+   */
+  private static void createFlattenedListFromMap(List<Distributable> blockInfos,
+      List<NodeBlockRelation> flattenedList, Set<Distributable> uniqueBlocks,
+      Set<String> nodeList) {
     for (Distributable blockInfo : blockInfos) {
+      // put the blocks in the set
+      uniqueBlocks.add(blockInfo);
+
       try {
-        for (final String eachNode : blockInfo.getLocations()) {
-          if (node2Idx.containsKey(eachNode)) {
-            Integer idx = node2Idx.get(eachNode);
-            List<Distributable> blocks = node2Blocks.get(idx).getBlocks();
-            blocks.add(blockInfo);
-          } else {
-            // add blocks to this node for the first time
-            Integer idx = node2Idx.size();
-            List<Distributable> blocks = new ArrayList<>();
-            blocks.add(blockInfo);
-            node2Blocks.add(idx, new NodeMultiBlockRelation(eachNode, blocks));
-            node2Idx.put(eachNode, idx);
-          }
+        for (String eachNode : blockInfo.getLocations()) {
+          NodeBlockRelation nbr = new NodeBlockRelation(blockInfo, eachNode);
+          flattenedList.add(nbr);
+          nodeList.add(eachNode);
         }
       } catch (IOException e) {
         throw new RuntimeException("error getting location of block: " + blockInfo.toString(), e);
       }
     }
-
-    return node2Blocks;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7a11f8e5/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java b/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
deleted file mode 100644
index 9c66ada..0000000
--- a/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.util;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datastore.block.Distributable;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-
-import org.apache.commons.lang3.StringUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class CarbonLoaderUtilTest {
-  private final static LogService LOGGER
-      = LogServiceFactory.getLogService(CarbonLoaderUtilTest.class.getName());
-
-  private List<Distributable> generateBlocks() {
-    List<Distributable> blockInfos = new ArrayList<>();
-    String filePath = "/fakepath";
-    String blockId = "1";
-
-    String[] locations = new String[] { "host2", "host3" };
-    ColumnarFormatVersion version = ColumnarFormatVersion.V1;
-
-    TableBlockInfo tableBlockInfo1 = new TableBlockInfo(filePath + "_a", 0,
-        blockId, locations, 30 * 1024 * 1024, version, null);
-    blockInfos.add(tableBlockInfo1);
-
-    TableBlockInfo tableBlockInfo2 = new TableBlockInfo(filePath + "_b", 0,
-        blockId, locations, 40 * 1024 * 1024, version, null);
-    blockInfos.add(tableBlockInfo2);
-
-    TableBlockInfo tableBlockInfo3 = new TableBlockInfo(filePath + "_c", 0,
-        blockId, locations, 20 * 1024 * 1024, version, null);
-    blockInfos.add(tableBlockInfo3);
-
-    TableBlockInfo tableBlockInfo4 = new TableBlockInfo(filePath + "_d", 0,
-        blockId, locations, 1, version, null);
-    blockInfos.add(tableBlockInfo4);
-
-    TableBlockInfo tableBlockInfo5 = new TableBlockInfo(filePath + "_e", 0,
-        blockId, locations, 1, version, null);
-    blockInfos.add(tableBlockInfo5);
-
-    TableBlockInfo tableBlockInfo6 = new TableBlockInfo(filePath + "_f", 0,
-        blockId, locations, 1, version, null);
-    blockInfos.add(tableBlockInfo6);
-
-    TableBlockInfo tableBlockInfo7 = new TableBlockInfo(filePath + "_g", 0,
-        blockId, locations, 1, version, null);
-    blockInfos.add(tableBlockInfo7);
-    return blockInfos;
-  }
-
-  private List<String> generateExecutors() {
-    List<String> activeNodes = new ArrayList<>();
-    activeNodes.add("host1");
-    activeNodes.add("host2");
-    activeNodes.add("host3");
-    return activeNodes;
-  }
-
-  @Test
-  public void testNodeBlockMappingByDataSize() throws Exception {
-    List<Distributable> blockInfos = generateBlocks();
-    List<String> activeNodes = generateExecutors();
-
-    // the blocks are assigned by size, so the number of block for each node are different
-    Map<String, List<Distributable>> nodeMappingBySize =
-        CarbonLoaderUtil.nodeBlockMapping(blockInfos, -1, activeNodes,
-            CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_SIZE_FIRST);
-    LOGGER.info(convertMapListAsString(nodeMappingBySize));
-    Assert.assertEquals(3, nodeMappingBySize.size());
-    for (Map.Entry<String, List<Distributable>> entry : nodeMappingBySize.entrySet()) {
-      if (entry.getValue().size() == 1) {
-        // only contains the biggest block
-        Assert.assertEquals(40 * 1024 * 1024L,
-            ((TableBlockInfo) entry.getValue().get(0)).getBlockLength());
-      } else {
-        Assert.assertTrue(entry.getValue().size() > 1);
-      }
-    }
-
-    // the blocks are assigned by number, so the number of blocks for each node are nearly the same
-    Map<String, List<Distributable>> nodeMappingByNum =
-        CarbonLoaderUtil.nodeBlockMapping(blockInfos, -1, activeNodes,
-            CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_NUM_FIRST);
-    LOGGER.info(convertMapListAsString(nodeMappingByNum));
-    Assert.assertEquals(3, nodeMappingBySize.size());
-    for (Map.Entry<String, List<Distributable>> entry : nodeMappingByNum.entrySet()) {
-      Assert.assertTrue(entry.getValue().size() == blockInfos.size() / 3
-          || entry.getValue().size() == blockInfos.size() / 3 + 1);
-    }
-  }
-
-  private <K, T> String convertMapListAsString(Map<K, List<T>> mapList) {
-    StringBuffer sb = new StringBuffer();
-    for (Map.Entry<K, List<T>> entry : mapList.entrySet()) {
-      String key = entry.getKey().toString();
-      String value = StringUtils.join(entry.getValue(), ", ");
-      sb.append(key).append(" -- ").append(value).append(System.lineSeparator());
-    }
-    return sb.toString();
-  }
-}
\ No newline at end of file


[24/49] carbondata git commit: [CARBONDATA-1480]Min Max Index Example for DataMap

Posted by ja...@apache.org.
[CARBONDATA-1480]Min Max Index Example for DataMap

Datamap Example. Implementation of Min Max Index through Datamap. And Using the Index while prunning.

This closes #1359


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

Branch: refs/heads/carbonstore-rebase4
Commit: ef437592970bb4777ba8d930c8fea0a3ec19d0dd
Parents: 46851d8
Author: sounakr <so...@gmail.com>
Authored: Thu Sep 28 16:21:05 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 00:24:04 2018 +0800

----------------------------------------------------------------------
 .../core/datamap/DataMapStoreManager.java       |  16 +-
 .../carbondata/core/datamap/TableDataMap.java   |  18 +-
 .../carbondata/core/datamap/dev/DataMap.java    |  11 +-
 .../core/datamap/dev/DataMapWriter.java         |   3 +-
 .../indexstore/SegmentPropertiesFetcher.java    |  36 +++
 .../blockletindex/BlockletDataMap.java          |   5 +-
 .../blockletindex/BlockletDataMapFactory.java   |  32 ++-
 datamap/examples/pom.xml                        | 111 ++++++++++
 .../datamap/examples/BlockletMinMax.java        |  41 ++++
 .../datamap/examples/MinMaxDataMap.java         | 143 ++++++++++++
 .../datamap/examples/MinMaxDataMapFactory.java  | 114 ++++++++++
 .../datamap/examples/MinMaxDataWriter.java      | 221 +++++++++++++++++++
 .../examples/MinMaxIndexBlockDetails.java       |  77 +++++++
 .../MinMaxDataMapExample.scala                  |  77 +++++++
 .../testsuite/datamap/DataMapWriterSuite.scala  |   2 +-
 pom.xml                                         |   2 +
 .../datamap/DataMapWriterListener.java          |   4 +-
 .../store/writer/AbstractFactDataWriter.java    |   7 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |   3 +
 19 files changed, 894 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
index d30483a..90e5fff 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
@@ -26,6 +26,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -103,7 +104,7 @@ public final class DataMapStoreManager {
       tableDataMaps = new ArrayList<>();
     }
     TableDataMap dataMap = getTableDataMap(dataMapName, tableDataMaps);
-    if (dataMap != null) {
+    if (dataMap != null && dataMap.getDataMapName().equalsIgnoreCase(dataMapName)) {
       throw new RuntimeException("Already datamap exists in that path with type " + dataMapName);
     }
 
@@ -113,12 +114,15 @@ public final class DataMapStoreManager {
       DataMapFactory dataMapFactory = factoryClass.newInstance();
       dataMapFactory.init(identifier, dataMapName);
       BlockletDetailsFetcher blockletDetailsFetcher;
+      SegmentPropertiesFetcher segmentPropertiesFetcher = null;
       if (dataMapFactory instanceof BlockletDetailsFetcher) {
         blockletDetailsFetcher = (BlockletDetailsFetcher) dataMapFactory;
       } else {
         blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
       }
-      dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, blockletDetailsFetcher);
+      segmentPropertiesFetcher = (SegmentPropertiesFetcher) blockletDetailsFetcher;
+      dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, blockletDetailsFetcher,
+          segmentPropertiesFetcher);
     } catch (Exception e) {
       LOGGER.error(e);
       throw new RuntimeException(e);
@@ -128,11 +132,11 @@ public final class DataMapStoreManager {
     return dataMap;
   }
 
-  private TableDataMap getTableDataMap(String dataMapName,
-      List<TableDataMap> tableDataMaps) {
+  private TableDataMap getTableDataMap(String dataMapName, List<TableDataMap> tableDataMaps) {
     TableDataMap dataMap = null;
-    for (TableDataMap tableDataMap: tableDataMaps) {
-      if (tableDataMap.getDataMapName().equals(dataMapName)) {
+    for (TableDataMap tableDataMap : tableDataMaps) {
+      if (tableDataMap.getDataMapName().equals(dataMapName) || (!tableDataMap.getDataMapName()
+          .equals(""))) {
         dataMap = tableDataMap;
         break;
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index 6555d6c..a841f37 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -23,10 +23,12 @@ import java.util.List;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.events.Event;
@@ -47,15 +49,19 @@ public final class TableDataMap extends OperationEventListener {
 
   private BlockletDetailsFetcher blockletDetailsFetcher;
 
+  private SegmentPropertiesFetcher segmentPropertiesFetcher;
+
   /**
    * It is called to initialize and load the required table datamap metadata.
    */
   public TableDataMap(AbsoluteTableIdentifier identifier, String dataMapName,
-      DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher) {
+      DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher,
+      SegmentPropertiesFetcher segmentPropertiesFetcher) {
     this.identifier = identifier;
     this.dataMapName = dataMapName;
     this.dataMapFactory = dataMapFactory;
     this.blockletDetailsFetcher = blockletDetailsFetcher;
+    this.segmentPropertiesFetcher = segmentPropertiesFetcher;
   }
 
   /**
@@ -68,11 +74,13 @@ public final class TableDataMap extends OperationEventListener {
   public List<ExtendedBlocklet> prune(List<Segment> segments, FilterResolverIntf filterExp,
       List<PartitionSpec> partitions) throws IOException {
     List<ExtendedBlocklet> blocklets = new ArrayList<>();
+    SegmentProperties segmentProperties;
     for (Segment segment : segments) {
       List<Blocklet> pruneBlocklets = new ArrayList<>();
       List<DataMap> dataMaps = dataMapFactory.getDataMaps(segment);
+      segmentProperties = segmentPropertiesFetcher.getSegmentProperties(segment.getSegmentNo());
       for (DataMap dataMap : dataMaps) {
-        pruneBlocklets.addAll(dataMap.prune(filterExp, partitions));
+        pruneBlocklets.addAll(dataMap.prune(filterExp, segmentProperties, partitions));
       }
       blocklets.addAll(addSegmentId(blockletDetailsFetcher
           .getExtendedBlocklets(pruneBlocklets, segment), segment.getSegmentNo()));
@@ -124,7 +132,11 @@ public final class TableDataMap extends OperationEventListener {
     List<Blocklet> blocklets = new ArrayList<>();
     List<DataMap> dataMaps = dataMapFactory.getDataMaps(distributable);
     for (DataMap dataMap : dataMaps) {
-      blocklets.addAll(dataMap.prune(filterExp, partitions));
+      blocklets.addAll(
+          dataMap.prune(
+              filterExp,
+              segmentPropertiesFetcher.getSegmentProperties(distributable.getSegmentId()),
+              partitions));
     }
     for (Blocklet blocklet: blocklets) {
       ExtendedBlocklet detailedBlocklet =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
index f3642d6..434b371 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.core.datamap.dev;
 import java.io.IOException;
 import java.util.List;
 
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.memory.MemoryException;
@@ -35,16 +36,6 @@ public interface DataMap {
   void init(DataMapModel dataMapModel) throws MemoryException, IOException;
 
   /**
-   * Prune the datamap with filter expression. It returns the list of
-   * blocklets where these filters can exist.
-   *
-   * @param filterExp
-   * @return
-   */
-  List<Blocklet> prune(FilterResolverIntf filterExp);
-
-  // TODO Move this method to Abstract class
-  /**
    * Prune the datamap with filter expression and partition information. It returns the list of
    * blocklets where these filters can exist.
    *

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
index 28163d7..413eaa5 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
@@ -27,7 +27,7 @@ public interface DataMapWriter {
    *  Start of new block notification.
    *  @param blockId file name of the carbondata file
    */
-  void onBlockStart(String blockId);
+  void onBlockStart(String blockId, String blockPath);
 
   /**
    * End of block notification
@@ -45,7 +45,6 @@ public interface DataMapWriter {
    * @param blockletId sequence number of blocklet in the block
    */
   void onBlockletEnd(int blockletId);
-
   /**
    * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
    * DataMapMeta returned in DataMapFactory.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
new file mode 100644
index 0000000..ec2ae93
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.indexstore;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+
+/**
+ * Fetches the detailed segmentProperties which has more information to execute the query
+ */
+public interface SegmentPropertiesFetcher {
+
+  /**
+   * get the Segment properties based on the SegmentID.
+   * @param segmentId
+   * @return
+   * @throws IOException
+   */
+  SegmentProperties getSegmentProperties(String segmentId) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index 9ec7a46..4bd6ae7 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -624,8 +624,7 @@ public class BlockletDataMap implements DataMap, Cacheable {
     return false;
   }
 
-  @Override
-  public List<Blocklet> prune(FilterResolverIntf filterExp) {
+  private List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties) {
     if (unsafeMemoryDMStore.getRowCount() == 0) {
       return new ArrayList<>();
     }
@@ -685,7 +684,7 @@ public class BlockletDataMap implements DataMap, Cacheable {
       }
     }
     // Prune with filters if the partitions are existed in this datamap
-    return prune(filterExp);
+    return prune(filterExp, segmentProperties);
   }
 
   private boolean isCorrectUUID(String[] fileDetails, PartitionSpec spec) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index 5eb077f..89e61d2 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -31,14 +31,19 @@ import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.SegmentFileStore;
+import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.util.DataFileFooterConverter;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.events.Event;
 
@@ -50,13 +55,17 @@ import org.apache.hadoop.fs.RemoteIterator;
 /**
  * Table map for blocklet
  */
-public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFetcher {
+public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFetcher,
+    SegmentPropertiesFetcher {
 
   private AbsoluteTableIdentifier identifier;
 
   // segmentId -> list of index file
   private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
 
+  // segmentId -> SegmentProperties.
+  private Map<String, SegmentProperties> segmentPropertiesMap = new HashMap<>();
+
   private Cache<TableBlockIndexUniqueIdentifier, DataMap> cache;
 
   @Override
@@ -251,4 +260,25 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
     // TODO: pass SORT_COLUMNS into this class
     return null;
   }
+
+  @Override public SegmentProperties getSegmentProperties(String segmentId) throws IOException {
+    SegmentProperties segmentProperties = segmentPropertiesMap.get(segmentId);
+    if (segmentProperties == null) {
+      int[] columnCardinality;
+      List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+          getTableBlockIndexUniqueIdentifiers(segmentId);
+      DataFileFooterConverter fileFooterConverter = new DataFileFooterConverter();
+      List<DataFileFooter> indexInfo =
+          fileFooterConverter.getIndexInfo(tableBlockIndexUniqueIdentifiers.get(0).getFilePath());
+      for (DataFileFooter fileFooter : indexInfo) {
+        List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
+        if (segmentProperties == null) {
+          columnCardinality = fileFooter.getSegmentInfo().getColumnCardinality();
+          segmentProperties = new SegmentProperties(columnInTable, columnCardinality);
+        }
+      }
+      segmentPropertiesMap.put(segmentId, segmentProperties);
+    }
+    return segmentProperties;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/datamap/examples/pom.xml
----------------------------------------------------------------------
diff --git a/datamap/examples/pom.xml b/datamap/examples/pom.xml
new file mode 100644
index 0000000..6832e62
--- /dev/null
+++ b/datamap/examples/pom.xml
@@ -0,0 +1,111 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.carbondata</groupId>
+        <artifactId>carbondata-parent</artifactId>
+        <version>1.3.0-SNAPSHOT</version>
+        <relativePath>../../pom.xml</relativePath>
+    </parent>
+
+    <artifactId>carbondata-datamap-examples</artifactId>
+    <name>Apache CarbonData :: Datamap Examples</name>
+
+    <properties>
+        <dev.path>${basedir}/../../dev</dev.path>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.carbondata</groupId>
+            <artifactId>carbondata-spark2</artifactId>
+            <version>${project.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.apache.spark</groupId>
+                    <artifactId>spark-hive-thriftserver_2.10</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.spark</groupId>
+                    <artifactId>spark-repl_2.10</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.apache.spark</groupId>
+                    <artifactId>spark-sql_2.10</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-sql_${scala.binary.version}</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-hive-thriftserver_${scala.binary.version}</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-repl_${scala.binary.version}</artifactId>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <sourceDirectory>src/minmaxdatamap/main/java</sourceDirectory>
+        <resources>
+            <resource>
+                <directory>.</directory>
+                <includes>
+                    <include>CARBON_EXAMPLESLogResource.properties</include>
+                </includes>
+            </resource>
+        </resources>
+        <plugins>
+            <plugin>
+                <groupId>org.scala-tools</groupId>
+                <artifactId>maven-scala-plugin</artifactId>
+                <version>2.15.2</version>
+                <executions>
+                    <execution>
+                        <id>compile</id>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                        <phase>compile</phase>
+                    </execution>
+                    <execution>
+                        <phase>process-resources</phase>
+                        <goals>
+                            <goal>compile</goal>
+                        </goals>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <artifactId>maven-compiler-plugin</artifactId>
+                <configuration>
+                    <source>1.7</source>
+                    <target>1.7</target>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+
+</project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/BlockletMinMax.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/BlockletMinMax.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/BlockletMinMax.java
new file mode 100644
index 0000000..e6968fe
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/BlockletMinMax.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.apache.carbondata.datamap.examples;
+
+
+public class BlockletMinMax {
+  private byte[][] Min;
+
+  private byte[][] Max;
+
+  public byte[][] getMin() {
+    return Min;
+  }
+
+  public void setMin(byte[][] min) {
+    Min = min;
+  }
+
+  public byte[][] getMax() {
+    return Max;
+  }
+
+  public void setMax(byte[][] max) {
+    Max = max;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
new file mode 100644
index 0000000..2ad6327
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap.examples;
+
+import java.io.BufferedReader;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
+import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.scan.filter.FilterUtil;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+import com.google.gson.Gson;
+
+/**
+ * Datamap implementation for min max blocklet.
+ */
+public class MinMaxDataMap implements DataMap {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(MinMaxDataMap.class.getName());
+
+  public static final String NAME = "clustered.minmax.btree.blocklet";
+
+  private String filePath;
+
+  private MinMaxIndexBlockDetails[] readMinMaxDataMap;
+
+  @Override public void init(String filePath) throws MemoryException, IOException {
+    this.filePath = filePath;
+    CarbonFile[] listFiles = getCarbonMinMaxIndexFiles(filePath, "0");
+    for (int i = 0; i < listFiles.length; i++) {
+      readMinMaxDataMap = readJson(listFiles[i].getPath());
+    }
+  }
+
+  private CarbonFile[] getCarbonMinMaxIndexFiles(String filePath, String segmentId) {
+    String path = filePath.substring(0, filePath.lastIndexOf("/") + 1);
+    CarbonFile carbonFile = FileFactory.getCarbonFile(path);
+    return carbonFile.listFiles(new CarbonFileFilter() {
+      @Override public boolean accept(CarbonFile file) {
+        return file.getName().endsWith(".minmaxindex");
+      }
+    });
+  }
+
+  public MinMaxIndexBlockDetails[] readJson(String filePath) throws IOException {
+    Gson gsonObjectToRead = new Gson();
+    DataInputStream dataInputStream = null;
+    BufferedReader buffReader = null;
+    InputStreamReader inStream = null;
+    MinMaxIndexBlockDetails[] readMinMax = null;
+    AtomicFileOperations fileOperation =
+        new AtomicFileOperationsImpl(filePath, FileFactory.getFileType(filePath));
+
+    try {
+      if (!FileFactory.isFileExist(filePath, FileFactory.getFileType(filePath))) {
+        return null;
+      }
+      dataInputStream = fileOperation.openForRead();
+      inStream = new InputStreamReader(dataInputStream,
+          CarbonCommonConstants.CARBON_DEFAULT_STREAM_ENCODEFORMAT);
+      buffReader = new BufferedReader(inStream);
+      readMinMax = gsonObjectToRead.fromJson(buffReader, MinMaxIndexBlockDetails[].class);
+    } catch (IOException e) {
+      return null;
+    } finally {
+      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+    }
+    return readMinMax;
+  }
+
+  /**
+   * Block Prunning logic for Min Max DataMap.
+   *
+   * @param filterExp
+   * @param segmentProperties
+   * @return
+   */
+  @Override public List<Blocklet> prune(FilterResolverIntf filterExp,
+      SegmentProperties segmentProperties) {
+    List<Blocklet> blocklets = new ArrayList<>();
+
+    if (filterExp == null) {
+      for (int i = 0; i < readMinMaxDataMap.length; i++) {
+        blocklets.add(new Blocklet(readMinMaxDataMap[i].getFilePath(),
+            String.valueOf(readMinMaxDataMap[i].getBlockletId())));
+      }
+    } else {
+      FilterExecuter filterExecuter =
+          FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
+      int startIndex = 0;
+      while (startIndex < readMinMaxDataMap.length) {
+        BitSet bitSet = filterExecuter.isScanRequired(readMinMaxDataMap[startIndex].getMaxValues(),
+            readMinMaxDataMap[startIndex].getMinValues());
+        if (!bitSet.isEmpty()) {
+          blocklets.add(new Blocklet(readMinMaxDataMap[startIndex].getFilePath(),
+              String.valueOf(readMinMaxDataMap[startIndex].getBlockletId())));
+        }
+        startIndex++;
+      }
+    }
+    return blocklets;
+  }
+
+  @Override
+  public void clear() {
+    readMinMaxDataMap = null;
+  }
+
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
new file mode 100644
index 0000000..78544d3
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap.examples;
+
+import java.io.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.util.ByteUtil;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+import com.google.gson.Gson;
+
+public class MinMaxDataWriter implements DataMapWriter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(TableInfo.class.getName());
+
+  private byte[][] pageLevelMin, pageLevelMax;
+
+  private byte[][] blockletLevelMin, blockletLevelMax;
+
+  private Map<Integer, BlockletMinMax> blockMinMaxMap;
+
+  private String blockPath;
+
+
+  @Override public void onBlockStart(String blockId, String blockPath) {
+    pageLevelMax = null;
+    pageLevelMin = null;
+    blockletLevelMax = null;
+    blockletLevelMin = null;
+    blockMinMaxMap = null;
+    blockMinMaxMap = new HashMap<Integer, BlockletMinMax>();
+    this.blockPath = blockPath;
+  }
+
+  @Override public void onBlockEnd(String blockId) {
+    updateMinMaxIndex(blockId);
+  }
+
+  @Override public void onBlockletStart(int blockletId) {
+  }
+
+  @Override public void onBlockletEnd(int blockletId) {
+    updateBlockletMinMax(blockletId);
+  }
+
+  @Override
+  public void onPageAdded(int blockletId, int pageId, ColumnPage[] pages) {
+    // Calculate Min and Max value within this page.
+
+    // As part of example we are extracting Min Max values Manually. The same can be done from
+    // retrieving the page statistics. For e.g.
+
+    // if (pageLevelMin == null && pageLevelMax == null) {
+    //    pageLevelMin[1] = CarbonUtil.getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //        pages[0].getStatistics().getMin());
+    //    pageLevelMax[1] = CarbonUtil.getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //        pages[0].getStatistics().getMax());
+    //  } else {
+    //    if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(pageLevelMin[1], CarbonUtil
+    //        .getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //            pages[0].getStatistics().getMin())) > 0) {
+    //      pageLevelMin[1] = CarbonUtil.getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //          pages[0].getStatistics().getMin());
+    //    }
+    //    if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(pageLevelMax[1], CarbonUtil
+    //        .getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //            pages[0].getStatistics().getMax())) < 0) {
+    //      pageLevelMax[1] = CarbonUtil.getValueAsBytes(pages[0].getStatistics().getDataType(),
+    //          pages[0].getStatistics().getMax());
+    //    }
+
+    byte[] value = new byte[pages[0].getBytes(0).length - 2];
+    if (pageLevelMin == null && pageLevelMax == null) {
+      pageLevelMin = new byte[2][];
+      pageLevelMax = new byte[2][];
+
+      System.arraycopy(pages[0].getBytes(0), 2, value, 0, value.length);
+      pageLevelMin[1] = value;
+      pageLevelMax[1] = value;
+
+    } else {
+      for (int rowIndex = 0; rowIndex < pages[0].getPageSize(); rowIndex++) {
+        System.arraycopy(pages[0].getBytes(rowIndex), 2, value, 0, value.length);
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(pageLevelMin[1], value) > 0) {
+          pageLevelMin[1] = value;
+        }
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(pageLevelMax[1], value) < 0) {
+          pageLevelMax[1] = value;
+        }
+      }
+    }
+  }
+
+  private void updateBlockletMinMax(int blockletId) {
+    if (blockletLevelMax == null || blockletLevelMin == null) {
+      blockletLevelMax = new byte[2][];
+      blockletLevelMin = new byte[2][];
+      if (pageLevelMax != null || pageLevelMin != null) {
+        blockletLevelMin = pageLevelMin;
+        blockletLevelMax = pageLevelMax;
+      }
+    } else {
+      if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockletLevelMin[1], pageLevelMin[1]) > 0) {
+        blockletLevelMin = pageLevelMin;
+      }
+
+      if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockletLevelMax[1], pageLevelMax[1]) > 0) {
+        blockletLevelMax = pageLevelMax;
+      }
+    }
+    BlockletMinMax blockletMinMax = new BlockletMinMax();
+    blockletMinMax.setMax(blockletLevelMax);
+    blockletMinMax.setMin(blockletLevelMin);
+    blockMinMaxMap.put(blockletId, blockletMinMax);
+  }
+
+
+  public void updateMinMaxIndex(String blockId) {
+    constructMinMaxIndex(blockId);
+  }
+
+
+
+  /**
+   * Construct the Min Max Index.
+   * @param blockId
+   */
+  public void constructMinMaxIndex(String blockId) {
+    // construct Min and Max values of each Blocklets present inside a block.
+    List<MinMaxIndexBlockDetails> tempMinMaxIndexBlockDetails = null;
+    tempMinMaxIndexBlockDetails = loadBlockDetails();
+    try {
+      writeMinMaxIndexFile(tempMinMaxIndexBlockDetails, blockPath, blockId);
+    } catch (IOException ex) {
+      LOGGER.info(" Unable to write the file");
+    }
+  }
+
+  /**
+   * loadBlockDetails into the MinMaxIndexBlockDetails class.
+   */
+  private List<MinMaxIndexBlockDetails> loadBlockDetails() {
+    List<MinMaxIndexBlockDetails> minMaxIndexBlockDetails = new ArrayList<MinMaxIndexBlockDetails>();
+    MinMaxIndexBlockDetails tmpminMaxIndexBlockDetails = new MinMaxIndexBlockDetails();
+
+    for (int index = 0; index < blockMinMaxMap.size(); index++) {
+      tmpminMaxIndexBlockDetails.setMinValues(blockMinMaxMap.get(index).getMin());
+      tmpminMaxIndexBlockDetails.setMaxValues(blockMinMaxMap.get(index).getMax());
+      tmpminMaxIndexBlockDetails.setBlockletId(index);
+      tmpminMaxIndexBlockDetails.setFilePath(this.blockPath);
+      minMaxIndexBlockDetails.add(tmpminMaxIndexBlockDetails);
+    }
+    return minMaxIndexBlockDetails;
+  }
+
+  /**
+   * Write the data to a file. This is JSON format file.
+   * @param minMaxIndexBlockDetails
+   * @param blockPath
+   * @param blockId
+   * @throws IOException
+   */
+  public void writeMinMaxIndexFile(List<MinMaxIndexBlockDetails> minMaxIndexBlockDetails,
+      String blockPath, String blockId) throws IOException {
+    String filePath = blockPath.substring(0, blockPath.lastIndexOf(File.separator) + 1) + blockId
+        + ".minmaxindex";
+    BufferedWriter brWriter = null;
+    DataOutputStream dataOutStream = null;
+    try {
+      FileFactory.createNewFile(filePath, FileFactory.getFileType(filePath));
+      dataOutStream = FileFactory.getDataOutputStream(filePath, FileFactory.getFileType(filePath));
+      Gson gsonObjectToWrite = new Gson();
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutStream,
+          CarbonCommonConstants.CARBON_DEFAULT_STREAM_ENCODEFORMAT));
+      String minmaxIndexData = gsonObjectToWrite.toJson(minMaxIndexBlockDetails);
+      brWriter.write(minmaxIndexData);
+    } catch (IOException ioe) {
+      LOGGER.info("Error in writing minMaxindex file");
+    } finally {
+      if (null != brWriter) {
+        brWriter.flush();
+      }
+      if (null != dataOutStream) {
+        dataOutStream.flush();
+      }
+      CarbonUtil.closeStreams(brWriter, dataOutStream);
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
new file mode 100644
index 0000000..0596db5
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.datamap.examples;
+
+import java.io.Serializable;
+
+public class MinMaxIndexBlockDetails implements Serializable {
+  private static final long serialVersionUID = 1206104914911491724L;
+
+  /**
+   * Min value of a column of one blocklet Bit-Packed
+   */
+  private byte[][] minValues;
+
+  /**
+   * Max value of a columns of one blocklet Bit-Packed
+   */
+  private byte[][] maxValues;
+
+  /**
+   * filePath pointing to the block.
+   */
+  private String filePath;
+
+  /**
+   * BlockletID of the block.
+   */
+  private Integer BlockletId;
+
+
+  public byte[][] getMinValues() {
+    return minValues;
+  }
+
+  public void setMinValues(byte[][] minValues) {
+    this.minValues = minValues;
+  }
+
+  public byte[][] getMaxValues() {
+    return maxValues;
+  }
+
+  public void setMaxValues(byte[][] maxValues) {
+    this.maxValues = maxValues;
+  }
+
+  public String getFilePath() {
+    return filePath;
+  }
+
+  public void setFilePath(String filePath) {
+    this.filePath = filePath;
+  }
+
+  public Integer getBlockletId() {
+    return BlockletId;
+  }
+
+  public void setBlockletId(Integer blockletId) {
+    BlockletId = blockletId;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala b/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
new file mode 100644
index 0000000..0cfe410
--- /dev/null
+++ b/datamap/examples/src/minmaxdatamap/test/scala/minmaxdatamaptestcase/MinMaxDataMapExample.scala
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.datamap.examples
+
+import java.io.File
+
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.util.CarbonProperties
+
+object MinMaxDataMapExample {
+  def main(args: Array[String]): Unit = {
+
+    val rootPath = new File(this.getClass.getResource("/").getPath
+                            + "").getCanonicalPath
+    val storeLocation = s"$rootPath/dataMap/examples/target/store"
+    val warehouse = s"$rootPath/datamap/examples/target/warehouse"
+    val metastoredb = s"$rootPath/datamap/examples/target"
+
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+
+    import org.apache.spark.sql.CarbonSession._
+
+    val spark = SparkSession
+      .builder()
+      .master("local")
+      .appName("CarbonDataMapExample")
+      .config("spark.sql.warehouse.dir", warehouse)
+      .getOrCreateCarbonSession(storeLocation)
+
+    spark.sparkContext.setLogLevel("ERROR")
+    import spark.implicits._
+
+    // register datamap writer
+    DataMapStoreManager.getInstance().createAndRegisterDataMap(
+      AbsoluteTableIdentifier.from(storeLocation, "default", "carbonminmax"),
+      classOf[MinMaxDataMapFactory].getName,
+      MinMaxDataMap.NAME)
+
+    spark.sql("DROP TABLE IF EXISTS carbonminmax")
+
+    val df = spark.sparkContext.parallelize(1 to 33000)
+      .map(x => ("a", "b", x))
+      .toDF("c1", "c2", "c3")
+
+    // save dataframe to carbon file
+    df.write
+      .format("carbondata")
+      .option("tableName", "carbonminmax")
+      .mode(SaveMode.Overwrite)
+      .save()
+
+    // Query the table.
+    spark.sql("select c2 from carbonminmax").show(20, false)
+    spark.sql("select c2 from carbonminmax where c2 = 'b'").show(20, false)
+    spark.sql("DROP TABLE IF EXISTS carbonminmax")
+
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index c3f5d0a..c80ee2b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -191,7 +191,7 @@ object DataMapWriterSuite {
       callbackSeq :+= s"blocklet start $blockletId"
     }
 
-    override def onBlockStart(blockId: String): Unit = {
+    override def onBlockStart(blockId: String, blockPath: String): Unit = {
       callbackSeq :+= s"block start $blockId"
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 423f2dc..0b9917d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -437,6 +437,7 @@
         <module>format</module>
         <module>integration/spark2</module>
         <module>examples/spark2</module>
+        <module>datamap/examples</module>
         <module>integration/hive</module>
         <module>integration/presto</module>
         <module>examples/flink</module>
@@ -473,6 +474,7 @@
         <module>integration/presto</module>
         <module>streaming</module>
         <module>examples/spark2</module>
+        <module>datamap/examples</module>
       </modules>
       <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
index d739f8c..6fbbd3e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
@@ -83,10 +83,10 @@ public class DataMapWriterListener {
     LOG.info("DataMapWriter " + writer + " added");
   }
 
-  public void onBlockStart(String blockId) {
+  public void onBlockStart(String blockId, String blockPath) {
     for (List<DataMapWriter> writers : registry.values()) {
       for (DataMapWriter writer : writers) {
-        writer.onBlockStart(blockId);
+        writer.onBlockStart(blockId, blockPath);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
index c0b8065..02391cf 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -269,7 +269,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
 
   private void notifyDataMapBlockStart() {
     if (listener != null) {
-      listener.onBlockStart(carbonDataFileName);
+      listener.onBlockStart(carbonDataFileName, constructFactFileFullPath());
     }
   }
 
@@ -280,6 +280,11 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     blockletId = 0;
   }
 
+  private String constructFactFileFullPath() {
+    String factFilePath =
+        this.dataWriterVo.getCarbonDataDirectoryPath() + File.separator + this.carbonDataFileName;
+    return factFilePath;
+  }
   /**
    * Finish writing current file. It will flush stream, copy and rename temp file to final file
    * @param copyInCurrentThread set to false if want to do data copy in a new thread

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ef437592/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
index ddf444d..80d8154 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/v3/CarbonFactDataWriterImplV3.java
@@ -43,6 +43,7 @@ import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
 import org.apache.carbondata.processing.store.TablePage;
 import org.apache.carbondata.processing.store.writer.AbstractFactDataWriter;
 
+
 /**
  * Below class will be used to write the data in V3 format
  * <Column1 Data ChunkV3><Column1<Page1><Page2><Page3><Page4>>
@@ -157,6 +158,8 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
     }
   }
 
+
+
   /**
    * Write the collect blocklet data (blockletDataHolder) to file
    */


[41/49] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 11b3d43..527452a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -31,15 +31,14 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
@@ -63,21 +62,15 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    * entry count
    */
   private int entryCount;
-
   /**
    * return row
    */
-  private Object[] returnRow;
-  private int dimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
+  private IntermediateSortTempRow returnRow;
   private int readBufferSize;
   private String compressorName;
-  private Object[][] currentBuffer;
+  private IntermediateSortTempRow[] currentBuffer;
 
-  private Object[][] backupBuffer;
+  private IntermediateSortTempRow[] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -100,27 +93,21 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
   private int numberOfObjectRead;
 
-  private int nullSetWordsLength;
-
-  private Comparator<Object[]> comparator;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+  private Comparator<IntermediateSortTempRow> comparator;
   /**
    * Constructor to initialize
    */
   public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters parameters) {
     // set temp file
     this.tempFile = tempFile;
-    this.dimCnt = parameters.getDimColCount();
-    this.complexCnt = parameters.getComplexDimColCount();
-    this.measureCnt = parameters.getMeasureColCount();
-    this.isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
-    this.measureDataTypes = parameters.getMeasureDataType();
     this.readBufferSize = parameters.getBufferSize();
     this.compressorName = parameters.getSortTempCompressorName();
-
+    this.tableFieldStat = new TableFieldStat(parameters);
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
     this.executorService = Executors.newFixedThreadPool(1);
-    this.nullSetWordsLength = ((parameters.getMeasureColCount() - 1) >> 6) + 1;
-    comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
+    comparator = new IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn());
     initialize();
   }
 
@@ -169,11 +156,17 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    *
    * @throws CarbonSortKeyAndGroupByException problem while reading
    */
+  @Override
   public void readRow() throws CarbonSortKeyAndGroupByException {
     if (prefetch) {
       fillDataForPrefetch();
     } else {
-      this.returnRow = getRowFromStream();
+      try {
+        this.returnRow = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+        this.numberOfObjectRead++;
+      } catch (IOException e) {
+        throw new CarbonSortKeyAndGroupByException("Problems while reading row", e);
+      }
     }
   }
 
@@ -207,63 +200,22 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
   }
 
   /**
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
+   * get a batch of row, this interface is used in reading compressed sort temp files
+   *
+   * @param expected expected number in a batch
+   * @return a batch of row
+   * @throws IOException if error occurs while reading from stream
    */
-  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
-    Object[] row = new Object[dimCnt + measureCnt];
-    try {
-      int dimCount = 0;
-      for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
-        if (isNoDictionaryDimensionColumn[dimCount]) {
-          short aShort = stream.readShort();
-          byte[] col = new byte[aShort];
-          stream.readFully(col);
-          row[dimCount] = col;
-        } else {
-          int anInt = stream.readInt();
-          row[dimCount] = anInt;
-        }
-      }
-
-      // write complex dimensions here.
-      for (; dimCount < dimCnt; dimCount++) {
-        short aShort = stream.readShort();
-        byte[] col = new byte[aShort];
-        stream.readFully(col);
-        row[dimCount] = col;
-      }
-
-      long[] words = new long[nullSetWordsLength];
-      for (int i = 0; i < words.length; i++) {
-        words[i] = stream.readLong();
-      }
-
-      for (int mesCount = 0; mesCount < measureCnt; mesCount++) {
-        if (UnsafeCarbonRowPage.isSet(words, mesCount)) {
-          DataType dataType = measureDataTypes[mesCount];
-          if (dataType == DataTypes.SHORT) {
-            row[dimCount + mesCount] = stream.readShort();
-          } else if (dataType == DataTypes.INT) {
-            row[dimCount + mesCount] = stream.readInt();
-          } else if (dataType == DataTypes.LONG) {
-            row[dimCount + mesCount] = stream.readLong();
-          } else if (dataType == DataTypes.DOUBLE) {
-            row[dimCount + mesCount] = stream.readDouble();
-          } else if (DataTypes.isDecimal(dataType)) {
-            short aShort = stream.readShort();
-            byte[] bigDecimalInBytes = new byte[aShort];
-            stream.readFully(bigDecimalInBytes);
-            row[dimCount + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
-          } else {
-            throw new IllegalArgumentException("unsupported data type:" + dataType);
-          }
-        }
-      }
-      return row;
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException(e);
+  private IntermediateSortTempRow[] readBatchedRowFromStream(int expected)
+      throws IOException {
+    IntermediateSortTempRow[] holders = new IntermediateSortTempRow[expected];
+    for (int i = 0; i < expected; i++) {
+      IntermediateSortTempRow holder
+          = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+      holders[i] = holder;
     }
+    this.numberOfObjectRead += expected;
+    return holders;
   }
 
   /**
@@ -271,7 +223,7 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    *
    * @return row
    */
-  public Object[] getRow() {
+  public IntermediateSortTempRow getRow() {
     return this.returnRow;
   }
 
@@ -326,9 +278,7 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
   @Override public int hashCode() {
     int hash = 0;
-    hash += 31 * measureCnt;
-    hash += 31 * dimCnt;
-    hash += 31 * complexCnt;
+    hash += tableFieldStat.hashCode();
     hash += tempFile.hashCode();
     return hash;
   }
@@ -368,16 +318,12 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
   /**
    * This method will read the records from sort temp file and keep it in a buffer
    *
-   * @param numberOfRecords
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
+   * @param numberOfRecords number of records to be read
+   * @return batch of intermediate sort temp row
+   * @throws IOException if error occurs reading records from file
    */
-  private Object[][] prefetchRecordsFromFile(int numberOfRecords)
-      throws CarbonSortKeyAndGroupByException {
-    Object[][] records = new Object[numberOfRecords][];
-    for (int i = 0; i < numberOfRecords; i++) {
-      records[i] = getRowFromStream();
-    }
-    return records;
+  private IntermediateSortTempRow[] prefetchRecordsFromFile(int numberOfRecords)
+      throws IOException {
+    return readBatchedRowFromStream(numberOfRecords);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
index 4bbf61b..22673ff 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
@@ -21,25 +21,21 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
 import java.util.AbstractQueue;
-import java.util.Arrays;
 import java.util.PriorityQueue;
 import java.util.concurrent.Callable;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.SortTempChunkHolder;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeSortTempFileChunkHolder;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeIntermediateFileMerger implements Callable<Void> {
   /**
@@ -69,22 +65,13 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   private int totalNumberOfRecords;
 
   private SortParameters mergerParameters;
-
+  private TableFieldStat tableFieldStat;
   private File[] intermediateFiles;
-
   private File outPutFile;
 
-  private int dimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
   private int writeBufferSize;
   private String compressorName;
-
-  private long[] nullSetWords;
-
-  private ByteBuffer rowData;
+  private SortStepRowHandler sortStepRowHandler;
 
   private Throwable throwable;
 
@@ -97,16 +84,10 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
     this.fileCounter = intermediateFiles.length;
     this.intermediateFiles = intermediateFiles;
     this.outPutFile = outPutFile;
-    this.dimCnt = mergerParameters.getDimColCount();
-    this.complexCnt = mergerParameters.getComplexDimColCount();
-    this.measureCnt = mergerParameters.getMeasureColCount();
-    this.isNoDictionaryDimensionColumn = mergerParameters.getNoDictionaryDimnesionColumn();
-    this.measureDataTypes = mergerParameters.getMeasureDataType();
     this.writeBufferSize = mergerParameters.getBufferSize();
     this.compressorName = mergerParameters.getSortTempCompressorName();
-    this.nullSetWords = new long[((measureCnt - 1) >> 6) + 1];
-    // Take size of 2 MB for each row. I think it is high enough to use
-    rowData = ByteBuffer.allocate(2 * 1024 * 1024);
+    this.tableFieldStat = new TableFieldStat(mergerParameters);
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
   }
 
   @Override public Void call() throws Exception {
@@ -165,13 +146,14 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted record from file
+   * This method will be used to get sorted sort temp row from the sort temp files
    *
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] getSortedRecordFromFile() throws CarbonSortKeyAndGroupByException {
-    Object[] row = null;
+  private IntermediateSortTempRow getSortedRecordFromFile()
+      throws CarbonSortKeyAndGroupByException {
+    IntermediateSortTempRow row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will
@@ -235,7 +217,7 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
       this.recordHolderHeap.add(sortTempFileChunkHolder);
     }
 
-    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
+    LOGGER.info("Heap Size: " + this.recordHolderHeap.size());
   }
 
   /**
@@ -250,12 +232,12 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted row
+   * This method will be used to get the sorted sort temp row
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] next() throws CarbonSortKeyAndGroupByException {
+  private IntermediateSortTempRow next() throws CarbonSortKeyAndGroupByException {
     return getSortedRecordFromFile();
   }
 
@@ -272,82 +254,16 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   /**
    * Below method will be used to write data to file
    *
-   * @throws CarbonSortKeyAndGroupByException problem while writing
+   * @throws IOException problem while writing
    */
-  private void writeDataToFile(Object[] row) throws CarbonSortKeyAndGroupByException, IOException {
-    int dimCount = 0;
-    int size = 0;
-    for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
-      if (isNoDictionaryDimensionColumn[dimCount]) {
-        byte[] col = (byte[]) row[dimCount];
-        rowData.putShort((short) col.length);
-        size += 2;
-        rowData.put(col);
-        size += col.length;
-      } else {
-        rowData.putInt((int) row[dimCount]);
-        size += 4;
-      }
-    }
-
-    // write complex dimensions here.
-    int dimensionSize = dimCnt + complexCnt;
-    for (; dimCount < dimensionSize; dimCount++) {
-      byte[] col = (byte[]) row[dimCount];
-      rowData.putShort((short)col.length);
-      size += 2;
-      rowData.put(col);
-      size += col.length;
-    }
-    Arrays.fill(nullSetWords, 0);
-    int nullSetSize = nullSetWords.length * 8;
-    int nullLoc = size;
-    size += nullSetSize;
-    for (int mesCount = 0; mesCount < measureCnt; mesCount++) {
-      Object value = row[mesCount + dimensionSize];
-      if (null != value) {
-        DataType dataType = measureDataTypes[mesCount];
-        if (dataType == DataTypes.SHORT) {
-          rowData.putShort(size, (Short) value);
-          size += 2;
-        } else if (dataType == DataTypes.INT) {
-          rowData.putInt(size, (Integer) value);
-          size += 4;
-        } else if (dataType == DataTypes.LONG) {
-          rowData.putLong(size, (Long) value);
-          size += 8;
-        } else if (dataType == DataTypes.DOUBLE) {
-          rowData.putDouble(size, (Double) value);
-          size += 8;
-        } else if (DataTypes.isDecimal(dataType)) {
-          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(((BigDecimal) value));
-          rowData.putShort(size, (short) bigDecimalInBytes.length);
-          size += 2;
-          for (int i = 0; i < bigDecimalInBytes.length; i++) {
-            rowData.put(size++, bigDecimalInBytes[i]);
-          }
-        }
-        UnsafeCarbonRowPage.set(nullSetWords, mesCount);
-      } else {
-        UnsafeCarbonRowPage.unset(nullSetWords, mesCount);
-      }
-    }
-    for (int i = 0; i < nullSetWords.length; i++) {
-      rowData.putLong(nullLoc, nullSetWords[i]);
-      nullLoc += 8;
-    }
-    byte[] rowBytes = new byte[size];
-    rowData.position(0);
-    rowData.get(rowBytes);
-    stream.write(rowBytes);
-    rowData.clear();
+  private void writeDataToFile(IntermediateSortTempRow row) throws IOException {
+    sortStepRowHandler.writeIntermediateSortTempRowToOutputStream(row, stream);
   }
 
   private void finish() throws CarbonSortKeyAndGroupByException {
     clear();
     try {
       CarbonUtil.deleteFiles(intermediateFiles);
-      rowData.clear();
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while deleting the intermediate files");
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
index ce118d9..64f3c25 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
@@ -29,7 +29,8 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
-import org.apache.carbondata.processing.loading.sort.SortStepRowUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.SortTempChunkHolder;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeFinalMergePageHolder;
@@ -55,7 +56,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   private AbstractQueue<SortTempChunkHolder> recordHolderHeapLocal;
 
   private SortParameters parameters;
-  private SortStepRowUtil sortStepRowUtil;
+  private SortStepRowHandler sortStepRowHandler;
   /**
    * tempFileLocation
    */
@@ -68,7 +69,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   public UnsafeSingleThreadFinalSortFilesMerger(SortParameters parameters,
       String[] tempFileLocation) {
     this.parameters = parameters;
-    this.sortStepRowUtil = new SortStepRowUtil(parameters);
+    this.sortStepRowHandler = new SortStepRowHandler(parameters);
     this.tempFileLocation = tempFileLocation;
     this.tableName = parameters.getTableName();
   }
@@ -108,9 +109,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       LOGGER.info("Started adding first record from each page");
       for (final UnsafeCarbonRowPage rowPage : rowPages) {
 
-        SortTempChunkHolder sortTempFileChunkHolder = new UnsafeInmemoryHolder(rowPage,
-            parameters.getDimColCount() + parameters.getComplexDimColCount() + parameters
-                .getMeasureColCount(), parameters.getNumberOfSortColumns());
+        SortTempChunkHolder sortTempFileChunkHolder = new UnsafeInmemoryHolder(rowPage);
 
         // initialize
         sortTempFileChunkHolder.readRow();
@@ -121,9 +120,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       for (final UnsafeInMemoryIntermediateDataMerger merger : merges) {
 
         SortTempChunkHolder sortTempFileChunkHolder =
-            new UnsafeFinalMergePageHolder(merger, parameters.getNoDictionarySortColumn(),
-                parameters.getDimColCount() + parameters.getComplexDimColCount() + parameters
-                    .getMeasureColCount());
+            new UnsafeFinalMergePageHolder(merger, parameters.getNoDictionarySortColumn());
 
         // initialize
         sortTempFileChunkHolder.readRow();
@@ -142,7 +139,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
         recordHolderHeapLocal.add(sortTempFileChunkHolder);
       }
 
-      LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
+      LOGGER.info("Heap Size: " + this.recordHolderHeapLocal.size());
     } catch (Exception e) {
       LOGGER.error(e);
       throw new CarbonDataWriterException(e);
@@ -180,12 +177,14 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   }
 
   /**
-   * This method will be used to get the sorted row
+   * This method will be used to get the sorted row in 3-parted format.
+   * The row will feed the following writer process step.
    *
    * @return sorted row
    */
   public Object[] next() {
-    return sortStepRowUtil.convertRow(getSortedRecordFromFile());
+    IntermediateSortTempRow sortTempRow =  getSortedRecordFromFile();
+    return sortStepRowHandler.convertIntermediateSortTempRowTo3Parted(sortTempRow);
   }
 
   /**
@@ -193,8 +192,8 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
    *
    * @return sorted record sorted record
    */
-  private Object[] getSortedRecordFromFile() throws CarbonDataWriterException {
-    Object[] row = null;
+  private IntermediateSortTempRow getSortedRecordFromFile() throws CarbonDataWriterException {
+    IntermediateSortTempRow row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index 751903a..ea11e22 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -389,7 +389,6 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
           noDictionarySortColumnMapping, 0, noDictionarySortColumnMapping.length);
     }
     sortParameters.setNoDictionarySortColumn(noDictionarySortColumnMapping);
-
     String[] sortTempFileLocation = CarbonDataProcessorUtil.arrayAppend(tempStoreLocation,
         CarbonCommonConstants.FILE_SEPARATOR, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
     finalMerger =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
index 04efa1f..c06819c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
@@ -21,7 +21,6 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.math.BigDecimal;
 import java.util.AbstractQueue;
 import java.util.PriorityQueue;
 import java.util.concurrent.Callable;
@@ -29,11 +28,9 @@ import java.util.concurrent.Callable;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 public class IntermediateFileMerger implements Callable<Void> {
@@ -68,17 +65,12 @@ public class IntermediateFileMerger implements Callable<Void> {
   private File[] intermediateFiles;
 
   private File outPutFile;
-  private int dimCnt;
-  private int noDictDimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
   private int writeBufferSize;
   private String compressorName;
 
   private Throwable throwable;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
   /**
    * IntermediateFileMerger Constructor
    */
@@ -88,14 +80,10 @@ public class IntermediateFileMerger implements Callable<Void> {
     this.fileCounter = intermediateFiles.length;
     this.intermediateFiles = intermediateFiles;
     this.outPutFile = outPutFile;
-    this.dimCnt = mergerParameters.getDimColCount();
-    this.noDictDimCnt = mergerParameters.getNoDictionaryCount();
-    this.complexCnt = mergerParameters.getComplexDimColCount();
-    this.measureCnt = mergerParameters.getMeasureColCount();
-    this.isNoDictionaryDimensionColumn = mergerParameters.getNoDictionaryDimnesionColumn();
-    this.measureDataTypes = mergerParameters.getMeasureDataType();
     this.writeBufferSize = mergerParameters.getBufferSize();
     this.compressorName = mergerParameters.getSortTempCompressorName();
+    this.tableFieldStat = new TableFieldStat(mergerParameters);
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
   }
 
   @Override public Void call() throws Exception {
@@ -154,13 +142,14 @@ public class IntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted record from file
+   * This method will be used to get the sorted sort temp row from sort temp file
    *
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] getSortedRecordFromFile() throws CarbonSortKeyAndGroupByException {
-    Object[] row = null;
+  private IntermediateSortTempRow getSortedRecordFromFile()
+      throws CarbonSortKeyAndGroupByException {
+    IntermediateSortTempRow row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will
@@ -227,7 +216,7 @@ public class IntermediateFileMerger implements Callable<Void> {
       this.recordHolderHeap.add(sortTempFileChunkHolder);
     }
 
-    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
+    LOGGER.info("Heap Size: " + this.recordHolderHeap.size());
   }
 
   /**
@@ -242,12 +231,12 @@ public class IntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted row
+   * This method will be used to get the sorted sort temp row
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] next() throws CarbonSortKeyAndGroupByException {
+  private IntermediateSortTempRow next() throws CarbonSortKeyAndGroupByException {
     return getSortedRecordFromFile();
   }
 
@@ -264,62 +253,10 @@ public class IntermediateFileMerger implements Callable<Void> {
   /**
    * Below method will be used to write data to file
    *
-   * @throws CarbonSortKeyAndGroupByException problem while writing
+   * @throws IOException problem while writing
    */
-  private void writeDataToFile(Object[] row) throws CarbonSortKeyAndGroupByException {
-    try {
-      int[] mdkArray = (int[]) row[0];
-      byte[][] nonDictArray = (byte[][]) row[1];
-      int mdkIndex = 0;
-      int nonDictKeyIndex = 0;
-      // write dictionary and non dictionary dimensions here.
-      for (boolean nodictinary : isNoDictionaryDimensionColumn) {
-        if (nodictinary) {
-          byte[] col = nonDictArray[nonDictKeyIndex++];
-          stream.writeShort(col.length);
-          stream.write(col);
-        } else {
-          stream.writeInt(mdkArray[mdkIndex++]);
-        }
-      }
-      // write complex
-      for (; nonDictKeyIndex < noDictDimCnt + complexCnt; nonDictKeyIndex++) {
-        byte[] col = nonDictArray[nonDictKeyIndex++];
-        stream.writeShort(col.length);
-        stream.write(col);
-      }
-      // write measure
-      int fieldIndex = 0;
-      for (int counter = 0; counter < measureCnt; counter++) {
-        if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
-          stream.write((byte) 1);
-          DataType dataType = measureDataTypes[counter];
-          if (dataType == DataTypes.BOOLEAN) {
-            stream.writeBoolean((boolean)NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.SHORT) {
-            stream.writeShort((short) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.INT) {
-            stream.writeInt((int) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.LONG) {
-            stream.writeLong((long) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.DOUBLE) {
-            stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (DataTypes.isDecimal(dataType)) {
-            byte[] bigDecimalInBytes = DataTypeUtil
-                .bigDecimalToByte((BigDecimal) NonDictionaryUtil.getMeasure(fieldIndex, row));
-            stream.writeInt(bigDecimalInBytes.length);
-            stream.write(bigDecimalInBytes);
-          } else {
-            throw new IllegalArgumentException("unsupported data type:" + dataType);
-          }
-        } else {
-          stream.write((byte) 0);
-        }
-        fieldIndex++;
-      }
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
-    }
+  private void writeDataToFile(IntermediateSortTempRow row) throws IOException {
+    sortStepRowHandler.writeIntermediateSortTempRowToOutputStream(row, stream);
   }
 
   private void finish() throws CarbonSortKeyAndGroupByException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
new file mode 100644
index 0000000..9b6d1e8
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.sort.sortdata;
+
+import java.util.Comparator;
+
+import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+
+/**
+ * This class is used as comparator for comparing intermediate sort temp row
+ */
+public class IntermediateSortTempRowComparator implements Comparator<IntermediateSortTempRow> {
+  /**
+   * isSortColumnNoDictionary whether the sort column is not dictionary or not
+   */
+  private boolean[] isSortColumnNoDictionary;
+
+  /**
+   * @param isSortColumnNoDictionary isSortColumnNoDictionary
+   */
+  public IntermediateSortTempRowComparator(boolean[] isSortColumnNoDictionary) {
+    this.isSortColumnNoDictionary = isSortColumnNoDictionary;
+  }
+
+  /**
+   * Below method will be used to compare two sort temp row
+   */
+  public int compare(IntermediateSortTempRow rowA, IntermediateSortTempRow rowB) {
+    int diff = 0;
+    int dictIndex = 0;
+    int nonDictIndex = 0;
+
+    for (boolean isNoDictionary : isSortColumnNoDictionary) {
+
+      if (isNoDictionary) {
+        byte[] byteArr1 = rowA.getNoDictSortDims()[nonDictIndex];
+        byte[] byteArr2 = rowB.getNoDictSortDims()[nonDictIndex];
+        nonDictIndex++;
+
+        int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
+        if (difference != 0) {
+          return difference;
+        }
+      } else {
+        int dimFieldA = rowA.getDictSortDims()[dictIndex];
+        int dimFieldB = rowB.getDictSortDims()[dictIndex];
+        dictIndex++;
+
+        diff = dimFieldA - dimFieldB;
+        if (diff != 0) {
+          return diff;
+        }
+      }
+    }
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
index d2579d2..3f94533 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
@@ -40,14 +40,11 @@ public class NewRowComparator implements Comparator<Object[]> {
    */
   public int compare(Object[] rowA, Object[] rowB) {
     int diff = 0;
-
     int index = 0;
 
     for (boolean isNoDictionary : noDictionarySortColumnMaping) {
-
       if (isNoDictionary) {
         byte[] byteArr1 = (byte[]) rowA[index];
-
         byte[] byteArr2 = (byte[]) rowB[index];
 
         int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
@@ -57,6 +54,7 @@ public class NewRowComparator implements Comparator<Object[]> {
       } else {
         int dimFieldA = (int) rowA[index];
         int dimFieldB = (int) rowB[index];
+
         diff = dimFieldA - dimFieldB;
         if (diff != 0) {
           return diff;
@@ -65,7 +63,6 @@ public class NewRowComparator implements Comparator<Object[]> {
 
       index++;
     }
-
     return diff;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
index e01b587..7538c92 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
@@ -29,7 +29,7 @@ public class NewRowComparatorForNormalDims implements Comparator<Object[]> {
   private int numberOfSortColumns;
 
   /**
-   * RowComparatorForNormalDims Constructor
+   * NewRowComparatorForNormalDims Constructor
    *
    * @param numberOfSortColumns
    */
@@ -46,7 +46,6 @@ public class NewRowComparatorForNormalDims implements Comparator<Object[]> {
     int diff = 0;
 
     for (int i = 0; i < numberOfSortColumns; i++) {
-
       int dimFieldA = (int)rowA[i];
       int dimFieldB = (int)rowB[i];
       diff = dimFieldA - dimFieldB;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
deleted file mode 100644
index 0ae0b93..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sort.sortdata;
-
-import java.nio.ByteBuffer;
-import java.util.Comparator;
-
-import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-
-public class RowComparator implements Comparator<Object[]> {
-  /**
-   * noDictionaryCount represent number of no dictionary cols
-   */
-  private int noDictionaryCount;
-
-  /**
-   * noDictionaryColMaping mapping of dictionary dimensions and no dictionary dimensions.
-   */
-  private boolean[] noDictionarySortColumnMaping;
-
-  /**
-   * @param noDictionarySortColumnMaping
-   * @param noDictionaryCount
-   */
-  public RowComparator(boolean[] noDictionarySortColumnMaping, int noDictionaryCount) {
-    this.noDictionaryCount = noDictionaryCount;
-    this.noDictionarySortColumnMaping = noDictionarySortColumnMaping;
-  }
-
-  /**
-   * Below method will be used to compare two mdkey
-   */
-  public int compare(Object[] rowA, Object[] rowB) {
-    int diff = 0;
-
-    int normalIndex = 0;
-    int noDictionaryindex = 0;
-
-    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
-
-      if (isNoDictionary) {
-        byte[] byteArr1 = (byte[]) rowA[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-
-        ByteBuffer buff1 = ByteBuffer.wrap(byteArr1);
-
-        // extract a high card dims from complete byte[].
-        NonDictionaryUtil
-            .extractSingleHighCardDims(byteArr1, noDictionaryindex, noDictionaryCount, buff1);
-
-        byte[] byteArr2 = (byte[]) rowB[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-
-        ByteBuffer buff2 = ByteBuffer.wrap(byteArr2);
-
-        // extract a high card dims from complete byte[].
-        NonDictionaryUtil
-            .extractSingleHighCardDims(byteArr2, noDictionaryindex, noDictionaryCount, buff2);
-
-        int difference = UnsafeComparer.INSTANCE.compareTo(buff1, buff2);
-        if (difference != 0) {
-          return difference;
-        }
-        noDictionaryindex++;
-      } else {
-        int dimFieldA = NonDictionaryUtil.getDimension(normalIndex, rowA);
-        int dimFieldB = NonDictionaryUtil.getDimension(normalIndex, rowB);
-        diff = dimFieldA - dimFieldB;
-        if (diff != 0) {
-          return diff;
-        }
-        normalIndex++;
-      }
-
-    }
-
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
deleted file mode 100644
index 0883ae1..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.sort.sortdata;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-
-/**
- * This class is used as comparator for comparing dims which are non high cardinality dims.
- * Here the dims will be in form of int[] (surrogates) so directly comparing the integers.
- */
-public class RowComparatorForNormalDims implements Comparator<Object[]> {
-  /**
-   * dimension count
-   */
-  private int numberOfSortColumns;
-
-  /**
-   * RowComparatorForNormalDims Constructor
-   *
-   * @param numberOfSortColumns
-   */
-  public RowComparatorForNormalDims(int numberOfSortColumns) {
-    this.numberOfSortColumns = numberOfSortColumns;
-  }
-
-  /**
-   * Below method will be used to compare two surrogate keys
-   *
-   * @see Comparator#compare(Object, Object)
-   */
-  public int compare(Object[] rowA, Object[] rowB) {
-    int diff = 0;
-
-    for (int i = 0; i < numberOfSortColumns; i++) {
-
-      int dimFieldA = NonDictionaryUtil.getDimension(i, rowA);
-      int dimFieldB = NonDictionaryUtil.getDimension(i, rowB);
-
-      diff = dimFieldA - dimFieldB;
-      if (diff != 0) {
-        return diff;
-      }
-    }
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
index 88695b9..a4ac0ea 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
@@ -37,6 +37,8 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
@@ -71,12 +73,12 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    * tableName
    */
   private String tableName;
-
+  private SortParameters sortParameters;
+  private SortStepRowHandler sortStepRowHandler;
   /**
    * tempFileLocation
    */
   private String[] tempFileLocation;
-  private SortParameters sortParameters;
 
   private int maxThreadForSorting;
 
@@ -89,6 +91,7 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
     this.tempFileLocation = tempFileLocation;
     this.tableName = tableName;
     this.sortParameters = sortParameters;
+    this.sortStepRowHandler = new SortStepRowHandler(sortParameters);
     try {
       maxThreadForSorting = Integer.parseInt(CarbonProperties.getInstance()
           .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD,
@@ -107,8 +110,7 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    */
   public void startFinalMerge() throws CarbonDataWriterException {
     List<File> filesToMerge = getFilesToMergeSort();
-    if (filesToMerge.size() == 0)
-    {
+    if (filesToMerge.size() == 0) {
       LOGGER.info("No file to merge in final merge stage");
       return;
     }
@@ -125,11 +127,9 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
 
     // get all the merged files
     List<File> files = new ArrayList<File>(tempFileLocation.length);
-    for (String tempLoc : tempFileLocation)
-    {
+    for (String tempLoc : tempFileLocation) {
       File[] subFiles = new File(tempLoc).listFiles(fileFilter);
-      if (null != subFiles && subFiles.length > 0)
-      {
+      if (null != subFiles && subFiles.length > 0) {
         files.addAll(Arrays.asList(subFiles));
       }
     }
@@ -226,13 +226,14 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
   }
 
   /**
-   * This method will be used to get the sorted row
+   * This method will be used to get the sorted sort temp row from the sort temp files
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
   public Object[] next() {
-    return getSortedRecordFromFile();
+    IntermediateSortTempRow sortTempRow = getSortedRecordFromFile();
+    return sortStepRowHandler.convertIntermediateSortTempRowTo3Parted(sortTempRow);
   }
 
   /**
@@ -241,8 +242,8 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] getSortedRecordFromFile() throws CarbonDataWriterException {
-    Object[] row = null;
+  private IntermediateSortTempRow getSortedRecordFromFile() throws CarbonDataWriterException {
+    IntermediateSortTempRow row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
index 57a19bd..c7efbd9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.processing.sort.sortdata;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
-import java.math.BigDecimal;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
@@ -32,12 +32,10 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
@@ -69,7 +67,8 @@ public class SortDataRows {
   private Semaphore semaphore;
 
   private SortParameters parameters;
-
+  private SortStepRowHandler sortStepRowHandler;
+  private ThreadLocal<ByteBuffer> rowBuffer;
   private int sortBufferSize;
 
   private SortIntermediateFileMerger intermediateFileMerger;
@@ -79,7 +78,7 @@ public class SortDataRows {
   public SortDataRows(SortParameters parameters,
       SortIntermediateFileMerger intermediateFileMerger) {
     this.parameters = parameters;
-
+    this.sortStepRowHandler = new SortStepRowHandler(parameters);
     this.intermediateFileMerger = intermediateFileMerger;
 
     int batchSize = CarbonProperties.getInstance().getBatchSize();
@@ -87,6 +86,12 @@ public class SortDataRows {
     this.sortBufferSize = Math.max(parameters.getSortBufferSize(), batchSize);
     // observer of writing file in thread
     this.threadStatusObserver = new ThreadStatusObserver();
+    this.rowBuffer = new ThreadLocal<ByteBuffer>() {
+      @Override protected ByteBuffer initialValue() {
+        byte[] backedArray = new byte[2 * 1024 * 1024];
+        return ByteBuffer.wrap(backedArray);
+      }
+    };
   }
 
   /**
@@ -130,8 +135,7 @@ public class SortDataRows {
         semaphore.acquire();
         dataSorterAndWriterExecutorService.execute(new DataSorterAndWriter(recordHolderListLocal));
       } catch (InterruptedException e) {
-        LOGGER.error(e,
-            "exception occurred while trying to acquire a semaphore lock: ");
+        LOGGER.error(e, "exception occurred while trying to acquire a semaphore lock: ");
         throw new CarbonSortKeyAndGroupByException(e);
       }
       // create the new holder Array
@@ -158,7 +162,7 @@ public class SortDataRows {
         }
         intermediateFileMerger.startMergingIfPossible();
         Object[][] recordHolderListLocal = recordHolderList;
-        sizeLeft = sortBufferSize - entryCount ;
+        sizeLeft = sortBufferSize - entryCount;
         if (sizeLeft > 0) {
           System.arraycopy(rowBatch, 0, recordHolderListLocal, entryCount, sizeLeft);
         }
@@ -212,7 +216,6 @@ public class SortDataRows {
           locationChosen + File.separator + parameters.getTableName() +
               System.nanoTime() + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
       writeDataToFile(recordHolderList, this.entryCount, file);
-
     }
 
     startFileBasedMerge();
@@ -220,7 +223,7 @@ public class SortDataRows {
   }
 
   /**
-   * Below method will be used to write data to file
+   * Below method will be used to write data to sort temp file
    *
    * @throws CarbonSortKeyAndGroupByException problem while writing
    */
@@ -233,60 +236,9 @@ public class SortDataRows {
           parameters.getFileWriteBufferSize(), parameters.getSortTempCompressorName());
       // write number of entries to the file
       stream.writeInt(entryCountLocal);
-      int complexDimColCount = parameters.getComplexDimColCount();
-      int dimColCount = parameters.getDimColCount() + complexDimColCount;
-      DataType[] type = parameters.getMeasureDataType();
-      boolean[] noDictionaryDimnesionMapping = parameters.getNoDictionaryDimnesionColumn();
-      Object[] row = null;
       for (int i = 0; i < entryCountLocal; i++) {
-        // get row from record holder list
-        row = recordHolderList[i];
-        int dimCount = 0;
-        // write dictionary and non dictionary dimensions here.
-        for (; dimCount < noDictionaryDimnesionMapping.length; dimCount++) {
-          if (noDictionaryDimnesionMapping[dimCount]) {
-            byte[] col = (byte[]) row[dimCount];
-            stream.writeShort(col.length);
-            stream.write(col);
-          } else {
-            stream.writeInt((int)row[dimCount]);
-          }
-        }
-        // write complex dimensions here.
-        for (; dimCount < dimColCount; dimCount++) {
-          byte[] value = (byte[])row[dimCount];
-          stream.writeShort(value.length);
-          stream.write(value);
-        }
-        // as measures are stored in separate array.
-        for (int mesCount = 0;
-             mesCount < parameters.getMeasureColCount(); mesCount++) {
-          Object value = row[mesCount + dimColCount];
-          if (null != value) {
-            stream.write((byte) 1);
-            DataType dataType = type[mesCount];
-            if (dataType == DataTypes.BOOLEAN) {
-              stream.writeBoolean((boolean) value);
-            } else if (dataType == DataTypes.SHORT) {
-              stream.writeShort((Short) value);
-            } else if (dataType == DataTypes.INT) {
-              stream.writeInt((Integer) value);
-            } else if (dataType == DataTypes.LONG) {
-              stream.writeLong((Long) value);
-            } else if (dataType == DataTypes.DOUBLE) {
-              stream.writeDouble((Double) value);
-            } else if (DataTypes.isDecimal(dataType)) {
-              BigDecimal val = (BigDecimal) value;
-              byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
-              stream.writeInt(bigDecimalInBytes.length);
-              stream.write(bigDecimalInBytes);
-            } else {
-              throw new IllegalArgumentException("unsupported data type:" + type[mesCount]);
-            }
-          } else {
-            stream.write((byte) 0);
-          }
-        }
+        sortStepRowHandler.writeRawRowAsIntermediateSortTempRowToOutputStream(
+            recordHolderList[i], stream, rowBuffer.get());
       }
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
@@ -301,7 +253,7 @@ public class SortDataRows {
    *
    * @throws CarbonSortKeyAndGroupByException
    */
-  public void deleteSortLocationIfExists() throws CarbonSortKeyAndGroupByException {
+  private void deleteSortLocationIfExists() throws CarbonSortKeyAndGroupByException {
     CarbonDataProcessorUtil.deleteSortLocationIfExists(parameters.getTempFileLocation());
   }
 
@@ -380,7 +332,8 @@ public class SortDataRows {
         // intermediate merging of sort temp files will be triggered
         intermediateFileMerger.addFileToMerge(sortTempFile);
         LOGGER.info("Time taken to sort and write sort temp file " + sortTempFile + " is: " + (
-            System.currentTimeMillis() - startTime));
+            System.currentTimeMillis() - startTime) + ", sort temp file size in MB is "
+            + sortTempFile.length() * 0.1 * 10 / 1024 / 1024);
       } catch (Throwable e) {
         try {
           threadStatusObserver.notifyFailed(e);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
index d726539..7e221a7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
@@ -21,6 +21,7 @@ import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.Comparator;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -30,14 +31,11 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHolder> {
@@ -71,20 +69,13 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   /**
    * return row
    */
-  private Object[] returnRow;
-  private int dimCnt;
-  private int noDictDimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private boolean[] isNoDictionarySortColumn;
-  private DataType[] measureDataTypes;
+  private IntermediateSortTempRow returnRow;
   private int readBufferSize;
   private String compressorName;
 
-  private Object[][] currentBuffer;
+  private IntermediateSortTempRow[] currentBuffer;
 
-  private Object[][] backupBuffer;
+  private IntermediateSortTempRow[] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -104,7 +95,9 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
    * totalRecordFetch
    */
   private int totalRecordFetch;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+  private Comparator<IntermediateSortTempRow> comparator;
   /**
    * Constructor to initialize
    *
@@ -115,16 +108,12 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   public SortTempFileChunkHolder(File tempFile, SortParameters sortParameters, String tableName) {
     // set temp file
     this.tempFile = tempFile;
-    this.dimCnt = sortParameters.getDimColCount();
-    this.noDictDimCnt = sortParameters.getNoDictionaryCount();
-    this.complexCnt = sortParameters.getComplexDimColCount();
-    this.measureCnt = sortParameters.getMeasureColCount();
-    this.isNoDictionaryDimensionColumn = sortParameters.getNoDictionaryDimnesionColumn();
-    this.isNoDictionarySortColumn = sortParameters.getNoDictionarySortColumn();
-    this.measureDataTypes = sortParameters.getMeasureDataType();
     this.readBufferSize = sortParameters.getBufferSize();
     this.compressorName = sortParameters.getSortTempCompressorName();
-
+    this.tableFieldStat = new TableFieldStat(sortParameters);
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
+    this.comparator = new IntermediateSortTempRowComparator(
+        tableFieldStat.getIsSortColNoDictFlags());
     this.executorService = Executors
         .newFixedThreadPool(1, new CarbonThreadFactory("SafeSortTempChunkHolderPool:" + tableName));
   }
@@ -178,7 +167,12 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
     if (prefetch) {
       fillDataForPrefetch();
     } else {
-      this.returnRow = getRowFromStream();
+      try {
+        this.returnRow = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+        this.numberOfObjectRead++;
+      } catch (IOException e) {
+        throw new CarbonSortKeyAndGroupByException("Problem while reading rows", e);
+      }
     }
   }
 
@@ -212,86 +206,28 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   }
 
   /**
-   * Reads row from file
+   * Read a batch of row from stream
+   *
    * @return Object[]
-   * @throws CarbonSortKeyAndGroupByException
+   * @throws IOException if error occurs while reading from stream
    */
-  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
-    // create new row of size 3 (1 for dims , 1 for high card , 1 for measures)
-
-    Object[] holder = new Object[3];
-    int index = 0;
-    int nonDicIndex = 0;
-    int[] dim = new int[dimCnt - noDictDimCnt];
-    byte[][] nonDicArray = new byte[noDictDimCnt + complexCnt][];
-    Object[] measures = new Object[measureCnt];
-    try {
-      // read dimension values
-      for (int i = 0; i < isNoDictionaryDimensionColumn.length; i++) {
-        if (isNoDictionaryDimensionColumn[i]) {
-          short len = stream.readShort();
-          byte[] array = new byte[len];
-          stream.readFully(array);
-          nonDicArray[nonDicIndex++] = array;
-        } else {
-          dim[index++] = stream.readInt();
-        }
-      }
-
-      for (int i = 0; i < complexCnt; i++) {
-        short len = stream.readShort();
-        byte[] array = new byte[len];
-        stream.readFully(array);
-        nonDicArray[nonDicIndex++] = array;
-      }
-
-      index = 0;
-      // read measure values
-      for (int i = 0; i < measureCnt; i++) {
-        if (stream.readByte() == 1) {
-          DataType dataType = measureDataTypes[i];
-          if (dataType == DataTypes.BOOLEAN) {
-            measures[index++] = stream.readBoolean();
-          } else if (dataType == DataTypes.SHORT) {
-            measures[index++] = stream.readShort();
-          } else if (dataType == DataTypes.INT) {
-            measures[index++] = stream.readInt();
-          } else if (dataType == DataTypes.LONG) {
-            measures[index++] = stream.readLong();
-          } else if (dataType == DataTypes.DOUBLE) {
-            measures[index++] = stream.readDouble();
-          } else if (DataTypes.isDecimal(dataType)) {
-            int len = stream.readInt();
-            byte[] buff = new byte[len];
-            stream.readFully(buff);
-            measures[index++] = DataTypeUtil.byteToBigDecimal(buff);
-          } else {
-            throw new IllegalArgumentException("unsupported data type:" + dataType);
-          }
-        } else {
-          measures[index++] = null;
-        }
-      }
-
-      NonDictionaryUtil.prepareOutObj(holder, dim, nonDicArray, measures);
-
-      // increment number if record read
-      this.numberOfObjectRead++;
-    } catch (IOException e) {
-      LOGGER.error("Problme while reading the madkey fom sort temp file");
-      throw new CarbonSortKeyAndGroupByException("Problem while reading the sort temp file ", e);
+  private IntermediateSortTempRow[] readBatchedRowFromStream(int expected) throws IOException {
+    IntermediateSortTempRow[] holders = new IntermediateSortTempRow[expected];
+    for (int i = 0; i < expected; i++) {
+      IntermediateSortTempRow holder
+          = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+      holders[i] = holder;
     }
-
-    //return out row
-    return holder;
+    this.numberOfObjectRead += expected;
+    return holders;
   }
 
   /**
-   * below method will be used to get the row
+   * below method will be used to get the sort temp row
    *
    * @return row
    */
-  public Object[] getRow() {
+  public IntermediateSortTempRow getRow() {
     return this.returnRow;
   }
 
@@ -330,31 +266,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   }
 
   @Override public int compareTo(SortTempFileChunkHolder other) {
-    int diff = 0;
-    int index = 0;
-    int noDictionaryIndex = 0;
-    int[] leftMdkArray = (int[]) returnRow[0];
-    int[] rightMdkArray = (int[]) other.returnRow[0];
-    byte[][] leftNonDictArray = (byte[][]) returnRow[1];
-    byte[][] rightNonDictArray = (byte[][]) other.returnRow[1];
-    for (boolean isNoDictionary : isNoDictionarySortColumn) {
-      if (isNoDictionary) {
-        diff = UnsafeComparer.INSTANCE
-            .compareTo(leftNonDictArray[noDictionaryIndex], rightNonDictArray[noDictionaryIndex]);
-        if (diff != 0) {
-          return diff;
-        }
-        noDictionaryIndex++;
-      } else {
-        diff = leftMdkArray[index] - rightMdkArray[index];
-        if (diff != 0) {
-          return diff;
-        }
-        index++;
-      }
-
-    }
-    return diff;
+    return comparator.compare(returnRow, other.getRow());
   }
 
   @Override public boolean equals(Object obj) {
@@ -372,9 +284,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
 
   @Override public int hashCode() {
     int hash = 0;
-    hash += 31 * measureCnt;
-    hash += 31 * dimCnt;
-    hash += 31 * complexCnt;
+    hash += tableFieldStat.hashCode();
     hash += tempFile.hashCode();
     return hash;
   }
@@ -414,16 +324,12 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   /**
    * This method will read the records from sort temp file and keep it in a buffer
    *
-   * @param numberOfRecords
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
+   * @param numberOfRecords number of records to be read
+   * @return batch of intermediate sort temp row
+   * @throws IOException if error occurs while reading reading records
    */
-  private Object[][] prefetchRecordsFromFile(int numberOfRecords)
-      throws CarbonSortKeyAndGroupByException {
-    Object[][] records = new Object[numberOfRecords][];
-    for (int i = 0; i < numberOfRecords; i++) {
-      records[i] = getRowFromStream();
-    }
-    return records;
+  private IntermediateSortTempRow[] prefetchRecordsFromFile(int numberOfRecords)
+      throws IOException {
+    return readBatchedRowFromStream(numberOfRecords);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
new file mode 100644
index 0000000..0d1303a
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
@@ -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.apache.carbondata.processing.sort.sortdata;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+
+/**
+ * This class is used to hold field information for a table during data loading. These information
+ * will be used to convert/construct/destruct row in sort process step. Because complex field is
+ * processed the same as no-dict-no-sort-simple-dimension, so we treat them as the same and use
+ * `no-dict-no-sort-dim` related variable to represent them in this class.
+ */
+public class TableFieldStat implements Serializable {
+  private static final long serialVersionUID = 201712070950L;
+  private int dictSortDimCnt = 0;
+  private int dictNoSortDimCnt = 0;
+  private int noDictSortDimCnt = 0;
+  private int noDictNoSortDimCnt = 0;
+  // whether sort column is of dictionary type or not
+  private boolean[] isSortColNoDictFlags;
+  private int measureCnt;
+  private DataType[] measureDataType;
+
+  // indices for dict & sort dimension columns
+  private int[] dictSortDimIdx;
+  // indices for dict & no-sort dimension columns
+  private int[] dictNoSortDimIdx;
+  // indices for no-dict & sort dimension columns
+  private int[] noDictSortDimIdx;
+  // indices for no-dict & no-sort dimension columns, including complex columns
+  private int[] noDictNoSortDimIdx;
+  // indices for measure columns
+  private int[] measureIdx;
+
+  public TableFieldStat(SortParameters sortParameters) {
+    int noDictDimCnt = sortParameters.getNoDictionaryCount();
+    int complexDimCnt = sortParameters.getComplexDimColCount();
+    int dictDimCnt = sortParameters.getDimColCount() - noDictDimCnt;
+    this.isSortColNoDictFlags = sortParameters.getNoDictionarySortColumn();
+    int sortColCnt = isSortColNoDictFlags.length;
+    for (boolean flag : isSortColNoDictFlags) {
+      if (flag) {
+        noDictSortDimCnt++;
+      } else {
+        dictSortDimCnt++;
+      }
+    }
+    this.measureCnt = sortParameters.getMeasureColCount();
+    this.measureDataType = sortParameters.getMeasureDataType();
+
+    // be careful that the default value is 0
+    this.dictSortDimIdx = new int[dictSortDimCnt];
+    this.dictNoSortDimIdx = new int[dictDimCnt - dictSortDimCnt];
+    this.noDictSortDimIdx = new int[noDictSortDimCnt];
+    this.noDictNoSortDimIdx = new int[noDictDimCnt + complexDimCnt - noDictSortDimCnt];
+    this.measureIdx = new int[measureCnt];
+
+    int tmpNoDictSortCnt = 0;
+    int tmpNoDictNoSortCnt = 0;
+    int tmpDictSortCnt = 0;
+    int tmpDictNoSortCnt = 0;
+    boolean[] isDimNoDictFlags = sortParameters.getNoDictionaryDimnesionColumn();
+
+    for (int i = 0; i < isDimNoDictFlags.length; i++) {
+      if (isDimNoDictFlags[i]) {
+        if (i < sortColCnt && isSortColNoDictFlags[i]) {
+          noDictSortDimIdx[tmpNoDictSortCnt++] = i;
+        } else {
+          noDictNoSortDimIdx[tmpNoDictNoSortCnt++] = i;
+        }
+      } else {
+        if (i < sortColCnt && !isSortColNoDictFlags[i]) {
+          dictSortDimIdx[tmpDictSortCnt++] = i;
+        } else {
+          dictNoSortDimIdx[tmpDictNoSortCnt++] = i;
+        }
+      }
+    }
+    dictNoSortDimCnt = tmpDictNoSortCnt;
+
+    int base = isDimNoDictFlags.length;
+    // adding complex dimension columns
+    for (int i = 0; i < complexDimCnt; i++) {
+      noDictNoSortDimIdx[tmpNoDictNoSortCnt++] = base + i;
+    }
+    noDictNoSortDimCnt = tmpNoDictNoSortCnt;
+
+    base += complexDimCnt;
+    // indices for measure columns
+    for (int i = 0; i < measureCnt; i++) {
+      measureIdx[i] = base + i;
+    }
+  }
+
+  public int getDictSortDimCnt() {
+    return dictSortDimCnt;
+  }
+
+  public int getDictNoSortDimCnt() {
+    return dictNoSortDimCnt;
+  }
+
+  public int getNoDictSortDimCnt() {
+    return noDictSortDimCnt;
+  }
+
+  public int getNoDictNoSortDimCnt() {
+    return noDictNoSortDimCnt;
+  }
+
+  public boolean[] getIsSortColNoDictFlags() {
+    return isSortColNoDictFlags;
+  }
+
+  public int getMeasureCnt() {
+    return measureCnt;
+  }
+
+  public DataType[] getMeasureDataType() {
+    return measureDataType;
+  }
+
+  public int[] getDictSortDimIdx() {
+    return dictSortDimIdx;
+  }
+
+  public int[] getDictNoSortDimIdx() {
+    return dictNoSortDimIdx;
+  }
+
+  public int[] getNoDictSortDimIdx() {
+    return noDictSortDimIdx;
+  }
+
+  public int[] getNoDictNoSortDimIdx() {
+    return noDictNoSortDimIdx;
+  }
+
+  public int[] getMeasureIdx() {
+    return measureIdx;
+  }
+
+  @Override public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof TableFieldStat)) return false;
+    TableFieldStat that = (TableFieldStat) o;
+    return dictSortDimCnt == that.dictSortDimCnt
+        && dictNoSortDimCnt == that.dictNoSortDimCnt
+        && noDictSortDimCnt == that.noDictSortDimCnt
+        && noDictNoSortDimCnt == that.noDictNoSortDimCnt
+        && measureCnt == that.measureCnt;
+  }
+
+  @Override public int hashCode() {
+    return Objects.hash(dictSortDimCnt, dictNoSortDimCnt, noDictSortDimCnt,
+        noDictNoSortDimCnt, measureCnt);
+  }
+}
\ No newline at end of file


[21/49] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
index a1ccab3..4293536 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
@@ -39,21 +39,19 @@ public class CarbonFormatDirectoryStructureTest {
    */
   @Test public void testTablePathStructure() throws IOException {
     CarbonTableIdentifier tableIdentifier = new CarbonTableIdentifier("d1", "t1", UUID.randomUUID().toString());
-    CarbonStorePath carbonStorePath = new CarbonStorePath(CARBON_STORE);
-    AbsoluteTableIdentifier absoluteTableIdentifier =
+    AbsoluteTableIdentifier identifier =
         AbsoluteTableIdentifier.from(CARBON_STORE + "/d1/t1", tableIdentifier);
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    assertTrue(carbonTablePath.getPath().replace("\\", "/").equals(CARBON_STORE + "/d1/t1"));
-    assertTrue(carbonTablePath.getSchemaFilePath().replace("\\", "/").equals(CARBON_STORE + "/d1/t1/Metadata/schema"));
-    assertTrue(carbonTablePath.getTableStatusFilePath().replace("\\", "/")
+    assertTrue(identifier.getTablePath().replace("\\", "/").equals(CARBON_STORE + "/d1/t1"));
+    assertTrue(CarbonTablePath.getSchemaFilePath(identifier.getTablePath()).replace("\\", "/").equals(CARBON_STORE + "/d1/t1/Metadata/schema"));
+    assertTrue(CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()).replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/tablestatus"));
-    assertTrue(carbonTablePath.getDictionaryFilePath("t1_c1").replace("\\", "/")
+    assertTrue(CarbonTablePath.getDictionaryFilePath(identifier.getTablePath(), "t1_c1").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.dict"));
-    assertTrue(carbonTablePath.getDictionaryMetaFilePath("t1_c1").replace("\\", "/")
+    assertTrue(CarbonTablePath.getDictionaryMetaFilePath(identifier.getTablePath(), "t1_c1").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.dictmeta"));
-    assertTrue(carbonTablePath.getSortIndexFilePath("t1_c1").replace("\\", "/")
+    assertTrue(CarbonTablePath.getSortIndexFilePath(identifier.getTablePath(),"t1_c1").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.sortindex"));
-    assertTrue(carbonTablePath.getCarbonDataFilePath("2", 3, 4L,  0, 0, "999").replace("\\", "/")
+    assertTrue(CarbonTablePath.getCarbonDataFilePath(identifier.getTablePath(), "2", 3, 4L,  0, 0, "999").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Fact/Part0/Segment_2/part-3-4_batchno0-0-999.carbondata"));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
deleted file mode 100644
index 91384c1..0000000
--- a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatSharedDictionaryTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.util.path;
-
-import java.io.IOException;
-
-import org.junit.Test;
-
-import static junit.framework.TestCase.assertTrue;
-
-/**
- * test shared dictionary paths
- */
-public class CarbonFormatSharedDictionaryTest {
-
-  private final String CARBON_STORE = "/opt/carbonstore";
-
-  /**
-   * test shared dictionary location
-   */
-  @Test public void testSharedDimentionLocation() throws IOException {
-    assertTrue(CarbonSharedDictionaryPath.getDictionaryFilePath(CARBON_STORE, "d1", "shared_c1").replace("\\", "/")
-        .equals(CARBON_STORE + "/d1/SharedDictionary/shared_c1.dict"));
-    assertTrue(CarbonSharedDictionaryPath.getDictionaryMetaFilePath(CARBON_STORE, "d1", "shared_c1").replace("\\", "/")
-        .equals(CARBON_STORE + "/d1/SharedDictionary/shared_c1.dictmeta"));
-    assertTrue(CarbonSharedDictionaryPath.getSortIndexFilePath(CARBON_STORE, "d1", "shared_c1").replace("\\", "/")
-        .equals(CARBON_STORE + "/d1/SharedDictionary/shared_c1.sortindex"));
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
index 308d041..d5500e1 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
@@ -44,7 +44,6 @@ import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
 import org.apache.carbondata.core.reader.CarbonDictionaryReaderImpl;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
 
@@ -72,7 +71,7 @@ public class CarbonDictionaryWriterImplTest {
 
   private String tableName;
 
-  private String carbonStorePath;
+  private String tablePath;
 
   private ColumnIdentifier columnIdentifier;
 
@@ -100,10 +99,10 @@ public class CarbonDictionaryWriterImplTest {
     init();
     this.databaseName = props.getProperty("database", "testSchema");
     this.tableName = props.getProperty("tableName", "carbon");
-    this.carbonStorePath = props.getProperty("storePath", "carbonStore");
+    this.tablePath = props.getProperty("storePath", "carbonStore");
     this.columnIdentifier = new ColumnIdentifier("Name", null, null);
     carbonTableIdentifier = new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    absoluteTableIdentifier = AbsoluteTableIdentifier.from(carbonStorePath, carbonTableIdentifier);
+    absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, carbonTableIdentifier);
     this.dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
             columnIdentifier.getDataType());
@@ -479,8 +478,8 @@ public class CarbonDictionaryWriterImplTest {
    * this method will delete the store path
    */
   private void deleteStorePath() {
-    FileFactory.FileType fileType = FileFactory.getFileType(this.carbonStorePath);
-    CarbonFile carbonFile = FileFactory.getCarbonFile(this.carbonStorePath, fileType);
+    FileFactory.FileType fileType = FileFactory.getFileType(this.tablePath);
+    CarbonFile carbonFile = FileFactory.getCarbonFile(this.tablePath, fileType);
     deleteRecursiveSilent(carbonFile);
   }
 
@@ -528,14 +527,12 @@ public class CarbonDictionaryWriterImplTest {
    * this method will form the dictionary directory paths
    */
   private void initDictionaryDirPaths() throws IOException {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(this.carbonStorePath, carbonTableIdentifier);
-    String dictionaryLocation = carbonTablePath.getMetadataDirectoryPath();
+    String dictionaryLocation = CarbonTablePath.getMetadataPath(tablePath);
     FileFactory.FileType fileType = FileFactory.getFileType(dictionaryLocation);
     if(!FileFactory.isFileExist(dictionaryLocation, fileType)) {
       FileFactory.mkdirs(dictionaryLocation, fileType);
     }
-    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier.getColumnId());
-    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(columnIdentifier.getColumnId());
+    this.dictionaryFilePath = CarbonTablePath.getDictionaryFilePath(tablePath, columnIdentifier.getColumnId());
+    this.dictionaryMetaFilePath = CarbonTablePath.getDictionaryMetaFilePath(tablePath, columnIdentifier.getColumnId());
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala
index 6ae87b9..40460c6 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonBatchSparkStreamingExample.scala
@@ -24,10 +24,6 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{CarbonEnv, SaveMode, SparkSession}
 import org.apache.spark.streaming.{Seconds, StreamingContext, Time}
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
-
 /**
  * This example introduces how to use CarbonData batch load to integrate
  * with Spark Streaming(it's DStream, not Spark Structured Streaming)
@@ -74,7 +70,6 @@ object CarbonBatchSparkStreamingExample {
            | """.stripMargin)
 
       val carbonTable = CarbonEnv.getCarbonTable(Some("default"), streamTableName)(spark)
-      val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       // batch load
       val path = s"$rootPath/examples/spark2/src/main/resources/streamSample.csv"
       spark.sql(
@@ -88,7 +83,7 @@ object CarbonBatchSparkStreamingExample {
       val serverSocket = new ServerSocket(7071)
       val thread1 = writeSocket(serverSocket)
       val thread2 = showTableCount(spark, streamTableName)
-      val ssc = startStreaming(spark, streamTableName, tablePath, checkpointPath)
+      val ssc = startStreaming(spark, streamTableName, checkpointPath)
       // wait for stop signal to stop Spark Streaming App
       waitForStopSignal(ssc)
       // it need to start Spark Streaming App in main thread
@@ -153,7 +148,7 @@ object CarbonBatchSparkStreamingExample {
   }
 
   def startStreaming(spark: SparkSession, tableName: String,
-      tablePath: CarbonTablePath, checkpointPath: String): StreamingContext = {
+      checkpointPath: String): StreamingContext = {
     var ssc: StreamingContext = null
     try {
       // recommend: the batch interval must set larger, such as 30s, 1min.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala
index 247a59b..bca7fa3 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonStructuredStreamingExample.scala
@@ -24,8 +24,9 @@ import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 // scalastyle:off println
 object CarbonStructuredStreamingExample {
@@ -89,7 +90,6 @@ object CarbonStructuredStreamingExample {
       }
 
       val carbonTable = CarbonEnv.getCarbonTable(Some("default"), streamTableName)(spark)
-      val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       // batch load
       val path = s"$rootPath/examples/spark2/src/main/resources/streamSample.csv"
       spark.sql(
@@ -101,7 +101,7 @@ object CarbonStructuredStreamingExample {
 
       // streaming ingest
       val serverSocket = new ServerSocket(7071)
-      val thread1 = startStreaming(spark, tablePath)
+      val thread1 = startStreaming(spark, carbonTable)
       val thread2 = writeSocket(serverSocket)
       val thread3 = showTableCount(spark, streamTableName)
 
@@ -152,7 +152,7 @@ object CarbonStructuredStreamingExample {
     thread
   }
 
-  def startStreaming(spark: SparkSession, tablePath: CarbonTablePath): Thread = {
+  def startStreaming(spark: SparkSession, carbonTable: CarbonTable): Thread = {
     val thread = new Thread() {
       override def run(): Unit = {
         var qry: StreamingQuery = null
@@ -167,7 +167,8 @@ object CarbonStructuredStreamingExample {
           qry = readSocketDF.writeStream
             .format("carbondata")
             .trigger(ProcessingTime("5 seconds"))
-            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("checkpointLocation",
+              CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath))
             .option("dbName", "default")
             .option("tableName", "stream_table")
             .start()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 087cf55..9dc8d38 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -70,7 +70,6 @@ import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeConverter;
 import org.apache.carbondata.core.util.DataTypeConverterImpl;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.BlockIndex;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
@@ -249,10 +248,6 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     }
   }
 
-  private static CarbonTablePath getTablePath(AbsoluteTableIdentifier absIdentifier) {
-    return CarbonStorePath.getCarbonTablePath(absIdentifier);
-  }
-
   /**
    * Set list of segments to access
    */
@@ -495,11 +490,10 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     List<InputSplit> splits = new ArrayList<InputSplit>();
     if (streamSegments != null && !streamSegments.isEmpty()) {
 
-      CarbonTablePath tablePath = CarbonStorePath.getCarbonTablePath(identifier);
       long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
       long maxSize = getMaxSplitSize(job);
       for (Segment segment : streamSegments) {
-        String segmentDir = tablePath.getSegmentDir(segment.getSegmentNo());
+        String segmentDir = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
         FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
         if (FileFactory.isFileExist(segmentDir, fileType)) {
           String indexName = CarbonTablePath.getCarbonStreamIndexFileName();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
index 3ef8afc..a4b3be8 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
@@ -37,7 +37,6 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.util.CarbonMetadataUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.FileHeader;
 import org.apache.carbondata.processing.loading.BadRecordsLogger;
@@ -127,9 +126,8 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
     maxCacheSize = hadoopConf.getInt(CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE,
         CarbonStreamOutputFormat.CARBON_STREAM_CACHE_SIZE_DEFAULT);
 
-    CarbonTablePath tablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
-    segmentDir = tablePath.getSegmentDir(segmentId);
+    segmentDir = CarbonTablePath.getSegmentPath(
+        carbonTable.getAbsoluteTableIdentifier().getTablePath(), segmentId);
     fileName = CarbonTablePath.getCarbonDataFileName(0, taskNo, 0, 0, "0");
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
index 423bb2a..dfa8dd1 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/SchemaReader.java
@@ -27,7 +27,7 @@ import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverte
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
@@ -37,8 +37,7 @@ public class SchemaReader {
 
   public static CarbonTable readCarbonTableFromStore(AbsoluteTableIdentifier identifier)
       throws IOException {
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(identifier);
-    String schemaFilePath = carbonTablePath.getSchemaFilePath();
+    String schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath());
     if (FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.LOCAL) ||
         FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.HDFS) ||
         FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.S3) ||
@@ -46,7 +45,7 @@ public class SchemaReader {
       String tableName = identifier.getCarbonTableIdentifier().getTableName();
 
       org.apache.carbondata.format.TableInfo tableInfo =
-          CarbonUtil.readSchemaFile(carbonTablePath.getSchemaFilePath());
+          CarbonUtil.readSchemaFile(CarbonTablePath.getSchemaFilePath(identifier.getTablePath()));
       SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
       TableInfo wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
           tableInfo,
@@ -63,22 +62,21 @@ public class SchemaReader {
   /**
    * the method returns the Wrapper TableInfo
    *
-   * @param absoluteTableIdentifier
+   * @param identifier
    * @return
    */
-  public static TableInfo getTableInfo(AbsoluteTableIdentifier absoluteTableIdentifier)
+  public static TableInfo getTableInfo(AbsoluteTableIdentifier identifier)
       throws IOException {
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     org.apache.carbondata.format.TableInfo thriftTableInfo =
-        CarbonUtil.readSchemaFile(carbonTablePath.getSchemaFilePath());
+        CarbonUtil.readSchemaFile(CarbonTablePath.getSchemaFilePath(identifier.getTablePath()));
     ThriftWrapperSchemaConverterImpl thriftWrapperSchemaConverter =
         new ThriftWrapperSchemaConverterImpl();
     CarbonTableIdentifier carbonTableIdentifier =
-        absoluteTableIdentifier.getCarbonTableIdentifier();
+        identifier.getCarbonTableIdentifier();
     return thriftWrapperSchemaConverter.fromExternalToWrapperTableInfo(
         thriftTableInfo,
         carbonTableIdentifier.getDatabaseName(),
         carbonTableIdentifier.getTableName(),
-        absoluteTableIdentifier.getTablePath());
+        identifier.getTablePath());
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
index ac17c4e..6365a6a 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
@@ -65,7 +65,6 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
@@ -188,11 +187,11 @@ public class StoreCreator {
   }
 
   public static CarbonTable createTable(
-      AbsoluteTableIdentifier absoluteTableIdentifier) throws IOException {
+      AbsoluteTableIdentifier identifier) throws IOException {
     TableInfo tableInfo = new TableInfo();
-    tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+    tableInfo.setDatabaseName(identifier.getCarbonTableIdentifier().getDatabaseName());
     TableSchema tableSchema = new TableSchema();
-    tableSchema.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    tableSchema.setTableName(identifier.getCarbonTableIdentifier().getTableName());
     List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
     ArrayList<Encoding> encodings = new ArrayList<>();
     encodings.add(Encoding.DICTIONARY);
@@ -284,13 +283,12 @@ public class StoreCreator {
     tableSchema.setSchemaEvalution(schemaEvol);
     tableSchema.setTableId(UUID.randomUUID().toString());
     tableInfo.setTableUniqueName(
-        absoluteTableIdentifier.getCarbonTableIdentifier().getTableUniqueName()
+        identifier.getCarbonTableIdentifier().getTableUniqueName()
     );
     tableInfo.setLastUpdatedTime(System.currentTimeMillis());
     tableInfo.setFactTable(tableSchema);
-    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath());
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String schemaFilePath = carbonTablePath.getSchemaFilePath();
+    tableInfo.setTablePath(identifier.getTablePath());
+    String schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath());
     String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
     CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
 
@@ -469,7 +467,7 @@ public class StoreCreator {
     loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
     listOfLoadFolderDetails.add(loadMetadataDetails);
 
-    String dataLoadLocation = schema.getCarbonTable().getMetaDataFilepath() + File.separator
+    String dataLoadLocation = schema.getCarbonTable().getMetadataPath() + File.separator
         + CarbonCommonConstants.LOADMETADATA_FILENAME;
 
     DataOutputStream dataOutputStream;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
index 5a2f831..8f7e88c 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
@@ -25,7 +25,6 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.service.impl.PathFactory;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.CarbonProjection;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
@@ -82,7 +81,8 @@ public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
         carbonTableReader.getCarbonCache(carbondataSplit.getSchemaTableName());
     checkNotNull(tableCacheModel, "tableCacheModel should not be null");
     checkNotNull(tableCacheModel.carbonTable, "tableCacheModel.carbonTable should not be null");
-    checkNotNull(tableCacheModel.tableInfo, "tableCacheModel.tableInfo should not be null");
+    checkNotNull(
+        tableCacheModel.carbonTable.getTableInfo(), "tableCacheModel.tableInfo should not be null");
 
     // Build Query Model
     CarbonTable targetTable = tableCacheModel.carbonTable;
@@ -92,8 +92,7 @@ public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
     try {
       Configuration conf = new Configuration();
       conf.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, "");
-      String carbonTablePath = PathFactory.getInstance()
-          .getCarbonTablePath(targetTable.getAbsoluteTableIdentifier(), null).getPath();
+      String carbonTablePath = targetTable.getAbsoluteTableIdentifier().getTablePath();
 
       conf.set(CarbonTableInputFormat.INPUT_DIR, carbonTablePath);
       JobConf jobConf = new JobConf(conf);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
index 8422c3e..4984406 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableCacheModel.java
@@ -17,27 +17,18 @@
 
 package org.apache.carbondata.presto.impl;
 
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.TableInfo;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
- * Caching metadata of CarbonData(e.g. TableIdentifier, TablePath, TableInfo, CarbonTable) in Class CarbonTableReader
+ * Caching metadata of CarbonData in Class CarbonTableReader
  * to speed up query
  */
 public class CarbonTableCacheModel {
 
-  public CarbonTableIdentifier carbonTableIdentifier;
-  public CarbonTablePath carbonTablePath;
-
-  public TableInfo tableInfo;
   public CarbonTable carbonTable;
 
   public boolean isValid() {
-    if (carbonTable != null && carbonTablePath != null && carbonTableIdentifier != null)
-      return true;
-    else return false;
+    return carbonTable != null;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
index b0271ef..09389f8 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
@@ -42,7 +42,6 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.service.impl.PathFactory;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
@@ -243,9 +242,10 @@ public class CarbonTableReader {
       updateSchemaList();
     }
     try {
-      if (isKeyExists && !FileFactory
-          .isFileExist(carbonCache.get().get(schemaTableName).carbonTablePath.getSchemaFilePath(),
-              fileType)) {
+      if (isKeyExists
+          && !FileFactory.isFileExist(
+              CarbonTablePath.getSchemaFilePath(
+                  carbonCache.get().get(schemaTableName).carbonTable.getTablePath()), fileType)) {
         removeTableFromCache(schemaTableName);
         throw new TableNotFoundException(schemaTableName);
       }
@@ -255,10 +255,12 @@ public class CarbonTableReader {
 
     if (isKeyExists) {
       CarbonTableCacheModel ctcm = carbonCache.get().get(schemaTableName);
-      if(ctcm != null && ctcm.tableInfo != null) {
-        Long latestTime = FileFactory.getCarbonFile(ctcm.carbonTablePath.getSchemaFilePath())
-            .getLastModifiedTime();
-        Long oldTime = ctcm.tableInfo.getLastUpdatedTime();
+      if(ctcm != null && ctcm.carbonTable.getTableInfo() != null) {
+        Long latestTime = FileFactory.getCarbonFile(
+            CarbonTablePath.getSchemaFilePath(
+                carbonCache.get().get(schemaTableName).carbonTable.getTablePath())
+        ).getLastModifiedTime();
+        Long oldTime = ctcm.carbonTable.getTableInfo().getLastUpdatedTime();
         if (DateUtils.truncate(new Date(latestTime), Calendar.MINUTE)
             .after(DateUtils.truncate(new Date(oldTime), Calendar.MINUTE))) {
           removeTableFromCache(schemaTableName);
@@ -312,21 +314,12 @@ public class CarbonTableReader {
 
       // Step 1: get store path of the table and cache it.
       // create table identifier. the table id is randomly generated.
-      cache.carbonTableIdentifier =
+      CarbonTableIdentifier carbonTableIdentifier =
               new CarbonTableIdentifier(table.getSchemaName(), table.getTableName(),
                       UUID.randomUUID().toString());
       String storePath = config.getStorePath();
-      String tablePath = storePath + "/" + cache.carbonTableIdentifier.getDatabaseName() + "/"
-          + cache.carbonTableIdentifier.getTableName();
-
-      // get the store path of the table.
-
-      AbsoluteTableIdentifier absoluteTableIdentifier =
-          AbsoluteTableIdentifier.from(tablePath, cache.carbonTableIdentifier);
-      cache.carbonTablePath =
-          PathFactory.getInstance().getCarbonTablePath(absoluteTableIdentifier, null);
-      // cache the table
-      carbonCache.get().put(table, cache);
+      String tablePath = storePath + "/" + carbonTableIdentifier.getDatabaseName() + "/"
+          + carbonTableIdentifier.getTableName();
 
       //Step 2: read the metadata (tableInfo) of the table.
       ThriftReader.TBaseCreator createTBase = new ThriftReader.TBaseCreator() {
@@ -338,7 +331,7 @@ public class CarbonTableReader {
         }
       };
       ThriftReader thriftReader =
-              new ThriftReader(cache.carbonTablePath.getSchemaFilePath(), createTBase);
+              new ThriftReader(CarbonTablePath.getSchemaFilePath(tablePath), createTBase);
       thriftReader.open();
       org.apache.carbondata.format.TableInfo tableInfo =
               (org.apache.carbondata.format.TableInfo) thriftReader.read();
@@ -355,9 +348,12 @@ public class CarbonTableReader {
       // Step 4: Load metadata info into CarbonMetadata
       CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo);
 
-      cache.tableInfo = wrapperTableInfo;
-      cache.carbonTable = CarbonMetadata.getInstance()
-              .getCarbonTable(cache.carbonTableIdentifier.getTableUniqueName());
+      cache.carbonTable = CarbonMetadata.getInstance().getCarbonTable(
+          table.getSchemaName(), table.getTableName());
+
+      // cache the table
+      carbonCache.get().put(table, cache);
+
       result = cache.carbonTable;
     } catch (Exception ex) {
       throw new RuntimeException(ex);
@@ -372,11 +368,10 @@ public class CarbonTableReader {
     List<CarbonLocalInputSplit> result = new ArrayList<>();
 
     CarbonTable carbonTable = tableCacheModel.carbonTable;
-    TableInfo tableInfo = tableCacheModel.tableInfo;
+    TableInfo tableInfo = tableCacheModel.carbonTable.getTableInfo();
     Configuration config = new Configuration();
     config.set(CarbonTableInputFormat.INPUT_SEGMENT_NUMBERS, "");
-    String carbonTablePath = PathFactory.getInstance()
-        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier(), null).getPath();
+    String carbonTablePath = carbonTable.getAbsoluteTableIdentifier().getTablePath();
     config.set(CarbonTableInputFormat.INPUT_DIR, carbonTablePath);
     config.set(CarbonTableInputFormat.DATABASE_NAME, carbonTable.getDatabaseName());
     config.set(CarbonTableInputFormat.TABLE_NAME, carbonTable.getTableName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index a41e738..1d7c791 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -52,7 +52,7 @@ import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolut
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata,
 CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter,
 CarbonDictionarySortIndexWriterImpl, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
@@ -323,10 +323,8 @@ object CarbonDataStoreCreator {
     )
     tableInfo.setLastUpdatedTime(System.currentTimeMillis())
     tableInfo.setFactTable(tableSchema)
-    val carbonTablePath: CarbonTablePath = CarbonStorePath.getCarbonTablePath(
-      absoluteTableIdentifier.getTablePath,
-      absoluteTableIdentifier.getCarbonTableIdentifier)
-    val schemaFilePath: String = carbonTablePath.getSchemaFilePath
+    val schemaFilePath: String = CarbonTablePath.getSchemaFilePath(
+      absoluteTableIdentifier.getTablePath)
     val schemaMetadataPath: String =
       CarbonTablePath.getFolderContainingFile(schemaFilePath)
     CarbonMetadata.getInstance.loadTableMetadata(tableInfo)
@@ -533,7 +531,7 @@ object CarbonDataStoreCreator {
       loadMetadataDetails.setLoadStartTime(
         loadMetadataDetails.getTimeStamp(readCurrentTime()))
       listOfLoadFolderDetails.add(loadMetadataDetails)
-      val dataLoadLocation: String = schema.getCarbonTable.getMetaDataFilepath + File.separator +
+      val dataLoadLocation: String = schema.getCarbonTable.getMetadataPath + File.separator +
                                      CarbonCommonConstants.LOADMETADATA_FILENAME
       val gsonObjectToWrite: Gson = new Gson()
       val writeOperation: AtomicFileOperations = new AtomicFileOperationsImpl(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
index 8e71257..ed6d741 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
@@ -18,18 +18,14 @@
 
 package org.apache.carbondata.cluster.sdv.generated
 
-import org.apache.spark.sql.CarbonEnv
 import org.apache.spark.sql.common.util._
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.writer.CarbonIndexFileMergeWriter
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 /**
  * Test Class for AlterTableTestCase to verify all scenerios

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index c84ae6b..a707c60 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterEach
 
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.spark.sql.test.util.QueryTest
@@ -48,13 +48,13 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
       datbaseName: String,
       tableName: String): Boolean = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(datbaseName, tableName)
-    val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
+    val partitionPath =
+      CarbonTablePath.getPartitionDir(carbonTable.getAbsoluteTableIdentifier.getTablePath)
     val fileType: FileFactory.FileType = FileFactory.getFileType(partitionPath)
     val carbonFile = FileFactory.getCarbonFile(partitionPath, fileType)
     val segments: ArrayBuffer[String] = ArrayBuffer()
     carbonFile.listFiles.foreach { file =>
-      segments += CarbonTablePath.DataPathUtil.getSegmentId(file.getAbsolutePath + "/dummy")
+      segments += CarbonTablePath.DataFileUtil.getSegmentId(file.getAbsolutePath + "/dummy")
     }
     segments.contains(segmentId)
   }
@@ -235,8 +235,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
     sql("create table stale(a string) stored by 'carbondata'")
     sql("insert into stale values('k')")
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "stale")
-    val tableStatusFile = new CarbonTablePath(null,
-      carbonTable.getTablePath).getTableStatusFilePath
+    val tableStatusFile = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath)
     FileFactory.getCarbonFile(tableStatusFile).delete()
     sql("insert into stale values('k')")
     checkAnswer(sql("select * from stale"), Row("k"))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
index 5cc4156..3c2fd71 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
@@ -29,7 +29,7 @@ import org.apache.carbondata.core.datastore.filesystem.CarbonFile
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
   var timeStampPropOrig: String = _
@@ -231,8 +231,7 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("insert overwrite table HiveOverwrite select * from THive")
     checkAnswer(sql("select count(*) from CarbonOverwrite"), sql("select count(*) from HiveOverwrite"))
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "carbonoverwrite")
-    val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
+    val partitionPath = CarbonTablePath.getPartitionDir(carbonTable.getAbsoluteTableIdentifier.getTablePath)
     val folder = new File(partitionPath)
     assert(folder.isDirectory)
     assert(folder.list().length == 1)
@@ -254,8 +253,7 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA local INPATH '$resourcesPath/100_olap.csv' overwrite INTO TABLE HiveOverwrite")
     checkAnswer(sql("select count(*) from TCarbonSourceOverwrite"), sql("select count(*) from HiveOverwrite"))
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "tcarbonsourceoverwrite")
-    val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
+    val partitionPath = CarbonTablePath.getPartitionDir(carbonTable.getAbsoluteTableIdentifier.getTablePath)
     val folder = new File(partitionPath)
 
     assert(folder.isDirectory)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
index 09268b5..8315848 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateTableAsSelect.scala
@@ -142,7 +142,7 @@ class TestCreateTableAsSelect extends QueryTest with BeforeAndAfterAll {
     val carbonTable = CarbonEnv.getInstance(Spark2TestQueryExecutor.spark).carbonMetastore
       .lookupRelation(Option("default"), "ctas_tblproperties_test")(Spark2TestQueryExecutor.spark)
       .asInstanceOf[CarbonRelation].carbonTable
-    val metadataFolderPath: CarbonFile = FileFactory.getCarbonFile(carbonTable.getMetaDataFilepath)
+    val metadataFolderPath: CarbonFile = FileFactory.getCarbonFile(carbonTable.getMetadataPath)
     assert(metadataFolderPath.exists())
     val dictFiles: Array[CarbonFile] = metadataFolderPath.listFiles(new CarbonFileFilter {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
index 2ef88a4..a7607c3 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
@@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.spark.sql.test.util.QueryTest
 
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
@@ -43,9 +43,7 @@ class DataCompactionLockTest extends QueryTest with BeforeAndAfterAll {
         new CarbonTableIdentifier(
           CarbonCommonConstants.DATABASE_DEFAULT_NAME, "compactionlocktesttable", "1")
       )
-  val carbonTablePath: CarbonTablePath = CarbonStorePath
-    .getCarbonTablePath(absoluteTableIdentifier)
-  val dataPath: String = carbonTablePath.getMetadataDirectoryPath
+  val dataPath: String = CarbonTablePath.getMetadataPath(absoluteTableIdentifier.getTablePath)
 
   val carbonLock: ICarbonLock =
     CarbonLockFactory

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
index 31a08fc..9afb890 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.datastore.TableSegmentUniqueIdentifier
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.hadoop.CacheClient
 import org.apache.spark.sql.test.util.QueryTest
 
@@ -112,11 +112,9 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "ignoremajor"
     )
-    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
 
-    val carbontablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
-      .getMetadataDirectoryPath
-    val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
+    val carbonTablePath = carbonTable.getMetadataPath
+    val segs = SegmentStatusManager.readLoadMetadata(carbonTablePath)
 
     // status should remain as compacted.
     assertResult(SegmentStatus.COMPACTED)(segs(3).getSegmentStatus)
@@ -134,9 +132,7 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       "ignoremajor"
     )
-    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    val carbontablePath = CarbonStorePath
-      .getCarbonTablePath(absoluteTableIdentifier).getMetadataDirectoryPath
+    val carbontablePath = carbonTable.getMetadataPath
     val segs = SegmentStatusManager.readLoadMetadata(carbontablePath)
 
     // status should remain as compacted for segment 2.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
index 42ac4df..68a3058 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
@@ -29,7 +29,7 @@ import org.apache.spark.sql.test.util.QueryTest
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
   var filePath: String = _
@@ -193,8 +193,7 @@ class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       tableName
     )
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath(segmentNo)
+    val segmentDir = carbonTable.getSemgentPath(segmentNo)
     new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
index db0a62c..b9d8e12 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
@@ -22,7 +22,7 @@ import java.io.{File, FilenameFilter}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.reader.CarbonIndexFileReader
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
@@ -48,8 +48,7 @@ class TestDataLoadWithFileName extends QueryTest with BeforeAndAfterAll {
     sql(s"LOAD DATA LOCAL INPATH '$testData' into table test_table_v3")
     val indexReader = new CarbonIndexFileReader()
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "test_table_v3")
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0")
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, "0")
     val carbonIndexPaths = new File(segmentDir)
       .listFiles(new FilenameFilter {
         override def accept(dir: File, name: String): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index 479db50..cbbb191 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -32,8 +32,8 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
 import org.apache.carbondata.core.metadata.CarbonMetadata
-import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
   var filePath: String = s"$resourcesPath/globalsort"
@@ -271,8 +271,7 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
     }
     sql(s"LOAD DATA LOCAL INPATH '$inputPath' INTO TABLE carbon_globalsort")
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "carbon_globalsort")
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getSegmentDir("0")
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, "0")
     assertResult(Math.max(7, defaultParallelism) + 1)(new File(segmentDir).listFiles().length)
   }
 
@@ -378,8 +377,7 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
 
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", tableName)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath(segmentNo)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentNo)
     new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
index ed58253..7c82f75 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
@@ -25,7 +25,7 @@ import org.apache.commons.lang3.time.DateUtils
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
@@ -79,8 +79,8 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
       "dataRetentionTable"
     )
     absoluteTableIdentifierForRetention = carbonTable2.getAbsoluteTableIdentifier
-    carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(absoluteTableIdentifierForRetention).getMetadataDirectoryPath
+    carbonTablePath = CarbonTablePath
+      .getMetadataPath(absoluteTableIdentifierForRetention.getTablePath)
     carbonTableStatusLock = CarbonLockFactory
       .getCarbonLockObj(absoluteTableIdentifierForLock, LockUsage.TABLE_STATUS_LOCK)
     carbonDeleteSegmentLock= CarbonLockFactory

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
index 0a21aed..e5de8da 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
@@ -61,9 +61,7 @@ class TestDataLoadingForPartitionTable extends QueryTest with BeforeAndAfterAll
 
   def validateDataFiles(tableUniqueName: String, segmentId: String, partitions: Seq[Int]): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
+    val segmentDir = carbonTable.getSemgentPath(segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
index c8f7be3..2ce46ef 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
@@ -72,8 +72,6 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
 
   def validateDataFiles(tableUniqueName: String, segmentId: String, partition: Int): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
     val partitions = CarbonFilters
       .getPartitions(Seq.empty,
         sqlContext.sparkSession,
@@ -334,9 +332,7 @@ class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfte
     sql(s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE mergeindexpartitionthree OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
 
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_mergeindexpartitionthree")
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-        carbonTable.getTablePath)
-    val details = SegmentStatusManager.readTableStatusFile(tablePath.getTableStatusFilePath)
+    val details = SegmentStatusManager.readTableStatusFile(CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath))
     val store = new SegmentFileStore(carbonTable.getTablePath, details(0).getSegmentFile)
     store.readIndexFiles()
     store.getIndexFiles

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
index 5fc7e3d..8adcb11 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
@@ -177,7 +177,7 @@ object CarbonStore {
     LOGGER.audit(s"Delete segment by Id request has been received for $dbName.$tableName")
     validateLoadIds(loadids)
 
-    val path = carbonTable.getMetaDataFilepath
+    val path = carbonTable.getMetadataPath
 
     try {
       val invalidLoadIds = SegmentStatusManager.updateDeletionStatus(
@@ -203,7 +203,7 @@ object CarbonStore {
     LOGGER.audit(s"Delete segment by Id request has been received for $dbName.$tableName")
 
     val time = validateTimeFormat(timestamp)
-    val path = carbonTable.getMetaDataFilepath
+    val path = carbonTable.getMetadataPath
 
     try {
       val invalidLoadTimestamps =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
index 32d121e..3dd9903 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonSparkFactory.scala
@@ -30,7 +30,7 @@ trait ColumnValidator {
  */
 trait DictionaryDetailService {
   def getDictionaryDetail(dictFolderPath: String, primDimensions: Array[CarbonDimension],
-      table: CarbonTableIdentifier, storePath: String): DictionaryDetail
+      tablePath: String): DictionaryDetail
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
index e861a8c..dbf47ab 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/DictionaryDetailHelper.scala
@@ -23,12 +23,11 @@ import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFi
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class DictionaryDetailHelper extends DictionaryDetailService {
-  def getDictionaryDetail(dictfolderPath: String, primDimensions: Array[CarbonDimension],
-      table: CarbonTableIdentifier, storePath: String): DictionaryDetail = {
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, table)
+  override def getDictionaryDetail(dictfolderPath: String, primDimensions: Array[CarbonDimension],
+      tablePath: String): DictionaryDetail = {
     val dictFilePaths = new Array[String](primDimensions.length)
     val dictFileExists = new Array[Boolean](primDimensions.length)
     val columnIdentifier = new Array[ColumnIdentifier](primDimensions.length)
@@ -50,7 +49,7 @@ class DictionaryDetailHelper extends DictionaryDetailService {
     // 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)
+      dictFilePaths(f._2) = CarbonTablePath.getDictionaryFilePath(tablePath, f._1.getColumnId)
       dictFileExists(f._2) =
         fileNamesMap.get(CarbonTablePath.getDictionaryFileName(f._1.getColumnId)) match {
           case None => false

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
index f2f4ecd..56a66b9 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.statusmanager.SegmentStatus
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.spark.util.GlobalDictionaryUtil
 
 /**
@@ -49,7 +49,7 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Par
  */
 class AlterTableAddColumnRDD[K, V](sc: SparkContext,
     @transient newColumns: Seq[ColumnSchema],
-    absoluteTableIdentifier: AbsoluteTableIdentifier)
+    identifier: AbsoluteTableIdentifier)
   extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
 
   val lockType: String = CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_TYPE,
@@ -70,8 +70,6 @@ class AlterTableAddColumnRDD[K, V](sc: SparkContext,
         // create dictionary file if it is a dictionary column
         if (columnSchema.hasEncoding(Encoding.DICTIONARY) &&
             !columnSchema.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          val carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier)
           var rawData: String = null
           if (null != columnSchema.getDefaultValue) {
             rawData = new String(columnSchema.getDefaultValue,
@@ -79,16 +77,15 @@ class AlterTableAddColumnRDD[K, V](sc: SparkContext,
           }
           CarbonProperties.getInstance.addProperty(CarbonCommonConstants.LOCK_TYPE, lockType)
           // Create table and metadata folders if not exist
-          val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+          val metadataDirectoryPath = CarbonTablePath.getMetadataPath(identifier.getTablePath)
           val fileType = FileFactory.getFileType(metadataDirectoryPath)
           if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
             FileFactory.mkdirs(metadataDirectoryPath, fileType)
           }
-          GlobalDictionaryUtil
-            .loadDefaultDictionaryValueForNewColumn(carbonTablePath,
-              columnSchema,
-              absoluteTableIdentifier,
-              rawData)
+          GlobalDictionaryUtil.loadDefaultDictionaryValueForNewColumn(
+            columnSchema,
+            identifier,
+            rawData)
         }
       } catch {
         case ex: Exception =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
index db29532..7acf4e2 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
@@ -40,10 +40,8 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
-import org.apache.carbondata.core.service.{CarbonCommonFactory, PathService}
 import org.apache.carbondata.core.statusmanager.SegmentStatus
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, CarbonUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
@@ -348,10 +346,6 @@ class CarbonGlobalDictionaryGenerateRDD(
         model.table,
         model.columnIdentifier(split.index),
         model.columnIdentifier(split.index).getDataType)
-      val pathService: PathService = CarbonCommonFactory.getPathService
-      val carbonTablePath: CarbonTablePath =
-        pathService
-          .getCarbonTablePath(model.table, dictionaryColumnUniqueIdentifier)
       if (StringUtils.isNotBlank(model.hdfsTempLocation)) {
         CarbonProperties.getInstance.addProperty(CarbonCommonConstants.HDFS_TEMP_LOCATION,
           model.hdfsTempLocation)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 94668bd..7815c99 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -50,7 +50,7 @@ import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
 import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.comparator.Comparator
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -635,7 +635,7 @@ object CommonUtil {
 
 
   def readLoadMetadataDetails(model: CarbonLoadModel): Unit = {
-    val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetaDataFilepath
+    val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetadataPath
     val details = SegmentStatusManager.readLoadMetadata(metadataPath)
     model.setLoadMetadataDetails(new util.ArrayList[LoadMetadataDetails](details.toList.asJava))
   }
@@ -866,20 +866,18 @@ object CommonUtil {
       val fileType = FileFactory.getFileType(databaseLocation)
       if (FileFactory.isFileExist(databaseLocation, fileType)) {
         val file = FileFactory.getCarbonFile(databaseLocation, fileType)
-          if (file.isDirectory) {
-            val tableFolders = file.listFiles()
-            tableFolders.foreach { tableFolder =>
-              if (tableFolder.isDirectory) {
-                val tablePath = databaseLocation +
-                                CarbonCommonConstants.FILE_SEPARATOR + tableFolder.getName
-                val identifier =
-                  AbsoluteTableIdentifier.from(tablePath, dbName, tableFolder.getName)
-                val carbonTablePath = CarbonStorePath.getCarbonTablePath(identifier)
-                val tableStatusFile = carbonTablePath.getTableStatusFilePath
-                if (FileFactory.isFileExist(tableStatusFile, fileType)) {
-                  val segmentStatusManager = new SegmentStatusManager(identifier)
-                  val carbonLock = segmentStatusManager.getTableStatusLock
-                  try {
+        if (file.isDirectory) {
+          val tableFolders = file.listFiles()
+          tableFolders.foreach { tableFolder =>
+            if (tableFolder.isDirectory) {
+              val tablePath = databaseLocation +
+                              CarbonCommonConstants.FILE_SEPARATOR + tableFolder.getName
+              val identifier =
+                AbsoluteTableIdentifier.from(tablePath, dbName, tableFolder.getName)
+              val tableStatusFile =
+                CarbonTablePath.getTableStatusFilePath(tablePath)
+              if (FileFactory.isFileExist(tableStatusFile, fileType)) {
+                try {
                   val carbonTable = CarbonMetadata.getInstance
                     .getCarbonTable(identifier.getCarbonTableIdentifier.getTableUniqueName)
                   DataLoadingUtil.deleteLoadsAndUpdateMetadata(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
index 6767ef7..cf35c12 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -372,7 +372,7 @@ object DataLoadingUtil {
       isForceDeletion: Boolean,
       carbonTable: CarbonTable,
       specs: util.List[PartitionSpec]): Unit = {
-    if (isLoadDeletionRequired(carbonTable.getMetaDataFilepath)) {
+    if (isLoadDeletionRequired(carbonTable.getMetadataPath)) {
       val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
 
       val (details, updationRequired) =
@@ -406,7 +406,7 @@ object DataLoadingUtil {
             }
             // read latest table status again.
             val latestMetadata = SegmentStatusManager
-              .readLoadMetadata(carbonTable.getMetaDataFilepath)
+              .readLoadMetadata(carbonTable.getMetadataPath)
 
             // update the metadata details from old to new status.
             val latestStatus = CarbonLoaderUtil
@@ -433,7 +433,7 @@ object DataLoadingUtil {
         if (updationCompletionStaus) {
           DeleteLoadFolders
             .physicalFactAndMeasureMetadataDeletion(absoluteTableIdentifier,
-              carbonTable.getMetaDataFilepath, isForceDeletion, specs)
+              carbonTable.getMetadataPath, isForceDeletion, specs)
         }
       }
     }
@@ -449,7 +449,7 @@ object DataLoadingUtil {
         absoluteTableIdentifier,
         isForceDeletion,
         details,
-        carbonTable.getMetaDataFilepath
+        carbonTable.getMetadataPath
       )
     (details, isUpdationRequired)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index 9e1ece7..2bd4f45 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -52,7 +52,7 @@ import org.apache.carbondata.core.reader.CarbonDictionaryReader
 import org.apache.carbondata.core.service.CarbonCommonFactory
 import org.apache.carbondata.core.statusmanager.SegmentStatus
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.csvinput.{CSVInputFormat, StringArrayWritable}
@@ -308,7 +308,7 @@ object GlobalDictionaryUtil {
     }
     val primDimensions = primDimensionsBuffer.map { x => x }.toArray
     val dictDetail = CarbonSparkFactory.getDictionaryDetailService.
-      getDictionaryDetail(dictFolderPath, primDimensions, table, carbonLoadModel.getTablePath)
+      getDictionaryDetail(dictFolderPath, primDimensions, carbonLoadModel.getTablePath)
     val dictFilePaths = dictDetail.dictFilePaths
     val dictFileExists = dictDetail.dictFileExists
     val columnIdentifier = dictDetail.columnIdentifiers
@@ -398,10 +398,6 @@ object GlobalDictionaryUtil {
     }
   }
 
-  // Hack for spark2 integration
-  var updateTableMetadataFunc: (CarbonLoadModel, SQLContext, DictionaryLoadModel,
-    Array[CarbonDimension]) => Unit = _
-
   /**
    * check whether global dictionary have been generated successfully or not
    *
@@ -705,10 +701,7 @@ object GlobalDictionaryUtil {
     try {
       val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
       val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
-      // create dictionary folder if not exists
-      val tablePath = carbonLoadModel.getTablePath
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier)
-      val dictfolderPath = carbonTablePath.getMetadataDirectoryPath
+      val dictfolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
       // columns which need to generate global dictionary file
       val dimensions = carbonTable.getDimensionByTableName(
         carbonTable.getTableName).asScala.toArray
@@ -845,12 +838,11 @@ object GlobalDictionaryUtil {
    * This method will write dictionary file, sortindex file and dictionary meta for new dictionary
    * column with default value
    *
-   * @param carbonTablePath
    * @param columnSchema
    * @param absoluteTableIdentifier
    * @param defaultValue
    */
-  def loadDefaultDictionaryValueForNewColumn(carbonTablePath: CarbonTablePath,
+  def loadDefaultDictionaryValueForNewColumn(
       columnSchema: ColumnSchema,
       absoluteTableIdentifier: AbsoluteTableIdentifier,
       defaultValue: String): Unit = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 20d3032..71ce2c6 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -33,12 +33,14 @@ import org.apache.carbondata.core.datamap.Segment
 import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.SegmentFileStore.SegmentFile
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes, DecimalType}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema._
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier, TableInfo, TableSchema}
 import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema, ParentColumnTableRelation}
 import org.apache.carbondata.core.service.CarbonCommonFactory
+import org.apache.carbondata.core.service.impl.ColumnUniqueIdGenerator
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentUpdateStatusManager}
 import org.apache.carbondata.core.util.DataTypeUtil
 import org.apache.carbondata.core.util.path.CarbonTablePath
@@ -208,8 +210,7 @@ class AlterTableColumnSchemaGenerator(
     alterTableModel: AlterTableAddColumnsModel,
     dbName: String,
     tableInfo: TableInfo,
-    carbonTablePath: CarbonTablePath,
-    tableIdentifier: CarbonTableIdentifier,
+    tableIdentifier: AbsoluteTableIdentifier,
     sc: SparkContext) {
 
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
@@ -370,7 +371,7 @@ object TableNewProcessor {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
     columnSchema.setEncodingList(encoders)
-    val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
+    val colUniqueIdGenerator = ColumnUniqueIdGenerator.getInstance
     val columnUniqueId = colUniqueIdGenerator.generateUniqueId(columnSchema)
     columnSchema.setColumnUniqueId(columnUniqueId)
     columnSchema.setColumnReferenceId(columnUniqueId)
@@ -434,7 +435,7 @@ class TableNewProcessor(cm: TableModel) {
       }
     }
     columnSchema.setEncodingList(encoders)
-    val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
+    val colUniqueIdGenerator = ColumnUniqueIdGenerator.getInstance
     val columnUniqueId = colUniqueIdGenerator.generateUniqueId(columnSchema)
     columnSchema.setColumnUniqueId(columnUniqueId)
     columnSchema.setColumnReferenceId(columnUniqueId)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
index 3c871db..1656efa 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
@@ -176,8 +176,6 @@ object PartitionUtils {
       getPartitionBlockList(identifier, segmentId, partitionIds, oldPartitionIds,
         partitionInfo, carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable).asScala
     val pathList: util.List[String] = new util.ArrayList[String]()
-    val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName, "")
-    val carbonTablePath = new CarbonTablePath(carbonTableIdentifier, tablePath)
     tableBlockInfoList.foreach{ tableBlockInfo =>
       val path = tableBlockInfo.getFilePath
       val timestamp = CarbonTablePath.DataFileUtil.getTimeStampFromFileName(path)
@@ -190,8 +188,8 @@ object PartitionUtils {
         val batchNo = CarbonTablePath.DataFileUtil.getBatchNoFromTaskNo(taskNo)
         val taskId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskNo)
         val bucketNumber = CarbonTablePath.DataFileUtil.getBucketNo(path)
-        val indexFilePath = carbonTablePath.getCarbonIndexFilePath(
-          String.valueOf(taskId), segmentId, batchNo, String.valueOf(bucketNumber),
+        val indexFilePath = CarbonTablePath.getCarbonIndexFilePath(
+          tablePath, String.valueOf(taskId), segmentId, batchNo, String.valueOf(bucketNumber),
           timestamp, version)
         // indexFilePath could be duplicated when multiple data file related to one index file
         if (indexFilePath != null && !pathList.contains(indexFilePath)) {


[35/49] carbondata git commit: Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row"

Posted by ja...@apache.org.
Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row"

This reverts commit de92ea9a123b17d903f2d1d4662299315c792954.


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

Branch: refs/heads/carbonstore-rebase4
Commit: 2058a4726c79b4c3f4ac396dc34ef109a3240531
Parents: 7a11f8e
Author: Jacky Li <ja...@qq.com>
Authored: Sat Feb 10 20:11:25 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:08:47 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/util/NonDictionaryUtil.java |  67 ++-
 .../presto/util/CarbonDataStoreCreator.scala    |   1 +
 .../load/DataLoadProcessorStepOnSpark.scala     |   6 +-
 .../loading/row/IntermediateSortTempRow.java    | 117 -----
 .../loading/sort/SortStepRowHandler.java        | 466 -------------------
 .../loading/sort/SortStepRowUtil.java           | 103 ++++
 .../sort/unsafe/UnsafeCarbonRowPage.java        | 331 +++++++++++--
 .../loading/sort/unsafe/UnsafeSortDataRows.java |  57 ++-
 .../unsafe/comparator/UnsafeRowComparator.java  |  95 ++--
 .../UnsafeRowComparatorForNormalDIms.java       |  59 +++
 .../UnsafeRowComparatorForNormalDims.java       |  59 ---
 .../sort/unsafe/holder/SortTempChunkHolder.java |   3 +-
 .../holder/UnsafeFinalMergePageHolder.java      |  19 +-
 .../unsafe/holder/UnsafeInmemoryHolder.java     |  21 +-
 .../holder/UnsafeSortTempFileChunkHolder.java   | 138 ++++--
 .../merger/UnsafeIntermediateFileMerger.java    | 118 ++++-
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  27 +-
 .../merger/CompactionResultSortProcessor.java   |   1 +
 .../sort/sortdata/IntermediateFileMerger.java   |  95 +++-
 .../IntermediateSortTempRowComparator.java      |  73 ---
 .../sort/sortdata/NewRowComparator.java         |   5 +-
 .../sortdata/NewRowComparatorForNormalDims.java |   3 +-
 .../processing/sort/sortdata/RowComparator.java |  94 ++++
 .../sortdata/RowComparatorForNormalDims.java    |  62 +++
 .../SingleThreadFinalSortFilesMerger.java       |  25 +-
 .../processing/sort/sortdata/SortDataRows.java  |  85 +++-
 .../sort/sortdata/SortTempFileChunkHolder.java  | 174 +++++--
 .../sort/sortdata/TableFieldStat.java           | 176 -------
 28 files changed, 1294 insertions(+), 1186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
index fca1244..d6ecfbc 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
@@ -82,26 +82,18 @@ public class NonDictionaryUtil {
   }
 
   /**
-   * Method to get the required dictionary Dimension from obj []
+   * Method to get the required Dimension from obj []
    *
    * @param index
    * @param row
    * @return
    */
-  public static int getDictDimension(int index, Object[] row) {
-    int[] dimensions = (int[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
+  public static Integer getDimension(int index, Object[] row) {
+
+    Integer[] dimensions = (Integer[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
+
     return dimensions[index];
-  }
 
-  /**
-   * Method to get the required non-dictionary & complex from 3-parted row
-   * @param index
-   * @param row
-   * @return
-   */
-  public static byte[] getNoDictOrComplex(int index, Object[] row) {
-    byte[][] nonDictArray = (byte[][]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-    return nonDictArray[index];
   }
 
   /**
@@ -116,11 +108,60 @@ public class NonDictionaryUtil {
     return measures[index];
   }
 
+  public static byte[] getByteArrayForNoDictionaryCols(Object[] row) {
+
+    return (byte[]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+  }
+
   public static void prepareOutObj(Object[] out, int[] dimArray, byte[][] byteBufferArr,
       Object[] measureArray) {
+
     out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray;
     out[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX] = byteBufferArr;
     out[WriteStepRowUtil.MEASURE] = measureArray;
 
   }
+
+  /**
+   * This method will extract the single dimension from the complete high card dims byte[].+     *
+   * The format of the byte [] will be,  Totallength,CompleteStartOffsets,Dat
+   *
+   * @param highCardArr
+   * @param index
+   * @param highCardinalityCount
+   * @param outBuffer
+   */
+  public static void extractSingleHighCardDims(byte[] highCardArr, int index,
+      int highCardinalityCount, ByteBuffer outBuffer) {
+    ByteBuffer buff = null;
+    short secIndex = 0;
+    short firstIndex = 0;
+    int length;
+    // if the requested index is a last one then we need to calculate length
+    // based on byte[] length.
+    if (index == highCardinalityCount - 1) {
+      // need to read 2 bytes(1 short) to determine starting offset and
+      // length can be calculated by array length.
+      buff = ByteBuffer.wrap(highCardArr, (index * 2) + 2, 2);
+    } else {
+      // need to read 4 bytes(2 short) to determine starting offset and
+      // length.
+      buff = ByteBuffer.wrap(highCardArr, (index * 2) + 2, 4);
+    }
+
+    firstIndex = buff.getShort();
+    // if it is a last dimension in high card then this will be last
+    // offset.so calculate length from total length
+    if (index == highCardinalityCount - 1) {
+      secIndex = (short) highCardArr.length;
+    } else {
+      secIndex = buff.getShort();
+    }
+
+    length = secIndex - firstIndex;
+
+    outBuffer.position(firstIndex);
+    outBuffer.limit(outBuffer.position() + length);
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index 7203278..1d7c791 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -383,6 +383,7 @@ object CarbonDataStoreCreator {
       .getInstance.createCache(CacheType.REVERSE_DICTIONARY)
 
     for (i <- set.indices) {
+      //      val dim = getDimension(dims, i).get
       val columnIdentifier: ColumnIdentifier =
         new ColumnIdentifier(dims.get(i).getColumnId, null, null)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
index 0422239..5124247 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
@@ -35,7 +35,7 @@ import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.loading.parser.impl.RowParserImpl
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler
+import org.apache.carbondata.processing.loading.sort.SortStepRowUtil
 import org.apache.carbondata.processing.loading.steps.DataWriterProcessorStepImpl
 import org.apache.carbondata.processing.sort.sortdata.SortParameters
 import org.apache.carbondata.processing.store.{CarbonFactHandler, CarbonFactHandlerFactory}
@@ -206,7 +206,7 @@ object DataLoadProcessorStepOnSpark {
     val model: CarbonLoadModel = modelBroadcast.value.getCopyWithTaskNo(index.toString)
     val conf = DataLoadProcessBuilder.createConfiguration(model)
     val sortParameters = SortParameters.createSortParameters(conf)
-    val sortStepRowHandler = new SortStepRowHandler(sortParameters)
+    val sortStepRowUtil = new SortStepRowUtil(sortParameters)
     TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) =>
       wrapException(e, model)
     }
@@ -216,7 +216,7 @@ object DataLoadProcessorStepOnSpark {
 
       override def next(): CarbonRow = {
         val row =
-          new CarbonRow(sortStepRowHandler.convertRawRowTo3Parts(rows.next().getData))
+          new CarbonRow(sortStepRowUtil.convertRow(rows.next().getData))
         rowCounter.add(1)
         row
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java b/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
deleted file mode 100644
index 8d351cf..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.loading.row;
-
-import java.nio.ByteBuffer;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.DataTypeUtil;
-
-/**
- * During sort procedure, each row will be written to sort temp file in this logic format.
- * an intermediate sort temp row consists 3 parts:
- * dictSort, noDictSort, noSortDimsAndMeasures(dictNoSort, noDictNoSort, measure)
- */
-public class IntermediateSortTempRow {
-  private int[] dictSortDims;
-  private byte[][] noDictSortDims;
-  private byte[] noSortDimsAndMeasures;
-
-  public IntermediateSortTempRow(int[] dictSortDims, byte[][] noDictSortDims,
-      byte[] noSortDimsAndMeasures) {
-    this.dictSortDims = dictSortDims;
-    this.noDictSortDims = noDictSortDims;
-    this.noSortDimsAndMeasures = noSortDimsAndMeasures;
-  }
-
-  public int[] getDictSortDims() {
-    return dictSortDims;
-  }
-
-  public byte[][] getNoDictSortDims() {
-    return noDictSortDims;
-  }
-
-  public byte[] getNoSortDimsAndMeasures() {
-    return noSortDimsAndMeasures;
-  }
-
-  /**
-   * deserialize from bytes array to get the no sort fields
-   * @param outDictNoSort stores the dict & no-sort fields
-   * @param outNoDictNoSort stores the no-dict & no-sort fields, including complex
-   * @param outMeasures stores the measure fields
-   * @param dataTypes data type for the measure
-   */
-  public void unpackNoSortFromBytes(int[] outDictNoSort, byte[][] outNoDictNoSort,
-      Object[] outMeasures, DataType[] dataTypes) {
-    ByteBuffer rowBuffer = ByteBuffer.wrap(noSortDimsAndMeasures);
-    // read dict_no_sort
-    int dictNoSortCnt = outDictNoSort.length;
-    for (int i = 0; i < dictNoSortCnt; i++) {
-      outDictNoSort[i] = rowBuffer.getInt();
-    }
-
-    // read no_dict_no_sort (including complex)
-    int noDictNoSortCnt = outNoDictNoSort.length;
-    for (int i = 0; i < noDictNoSortCnt; i++) {
-      short len = rowBuffer.getShort();
-      byte[] bytes = new byte[len];
-      rowBuffer.get(bytes);
-      outNoDictNoSort[i] = bytes;
-    }
-
-    // read measure
-    int measureCnt = outMeasures.length;
-    DataType tmpDataType;
-    Object tmpContent;
-    for (short idx = 0 ; idx < measureCnt; idx++) {
-      if ((byte) 0 == rowBuffer.get()) {
-        outMeasures[idx] = null;
-        continue;
-      }
-
-      tmpDataType = dataTypes[idx];
-      if (DataTypes.BOOLEAN == tmpDataType) {
-        if ((byte) 1 == rowBuffer.get()) {
-          tmpContent = true;
-        } else {
-          tmpContent = false;
-        }
-      } else if (DataTypes.SHORT == tmpDataType) {
-        tmpContent = rowBuffer.getShort();
-      } else if (DataTypes.INT == tmpDataType) {
-        tmpContent = rowBuffer.getInt();
-      } else if (DataTypes.LONG == tmpDataType) {
-        tmpContent = rowBuffer.getLong();
-      } else if (DataTypes.DOUBLE == tmpDataType) {
-        tmpContent = rowBuffer.getDouble();
-      } else if (DataTypes.isDecimal(tmpDataType)) {
-        short len = rowBuffer.getShort();
-        byte[] decimalBytes = new byte[len];
-        rowBuffer.get(decimalBytes);
-        tmpContent = DataTypeUtil.byteToBigDecimal(decimalBytes);
-      } else {
-        throw new IllegalArgumentException("Unsupported data type: " + tmpDataType);
-      }
-      outMeasures[idx] = tmpContent;
-    }
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
deleted file mode 100644
index f31a2b9..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
+++ /dev/null
@@ -1,466 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.sort;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.Serializable;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.apache.carbondata.core.memory.CarbonUnsafe;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
-
-/**
- * This class is used to convert/write/read row in sort step in carbondata.
- * It consists the following function:
- * 1. convert raw row & intermediate sort temp row to 3-parted row
- * 2. read/write intermediate sort temp row to sort temp file & unsafe memory
- * 3. write raw row directly to sort temp file & unsafe memory as intermediate sort temp row
- */
-public class SortStepRowHandler implements Serializable {
-  private static final long serialVersionUID = 1L;
-  private int dictSortDimCnt = 0;
-  private int dictNoSortDimCnt = 0;
-  private int noDictSortDimCnt = 0;
-  private int noDictNoSortDimCnt = 0;
-  private int measureCnt;
-
-  // indices for dict & sort dimension columns
-  private int[] dictSortDimIdx;
-  // indices for dict & no-sort dimension columns
-  private int[] dictNoSortDimIdx;
-  // indices for no-dict & sort dimension columns
-  private int[] noDictSortDimIdx;
-  // indices for no-dict & no-sort dimension columns, including complex columns
-  private int[] noDictNoSortDimIdx;
-  // indices for measure columns
-  private int[] measureIdx;
-
-  private DataType[] dataTypes;
-
-  /**
-   * constructor
-   * @param tableFieldStat table field stat
-   */
-  public SortStepRowHandler(TableFieldStat tableFieldStat) {
-    this.dictSortDimCnt = tableFieldStat.getDictSortDimCnt();
-    this.dictNoSortDimCnt = tableFieldStat.getDictNoSortDimCnt();
-    this.noDictSortDimCnt = tableFieldStat.getNoDictSortDimCnt();
-    this.noDictNoSortDimCnt = tableFieldStat.getNoDictNoSortDimCnt();
-    this.measureCnt = tableFieldStat.getMeasureCnt();
-    this.dictSortDimIdx = tableFieldStat.getDictSortDimIdx();
-    this.dictNoSortDimIdx = tableFieldStat.getDictNoSortDimIdx();
-    this.noDictSortDimIdx = tableFieldStat.getNoDictSortDimIdx();
-    this.noDictNoSortDimIdx = tableFieldStat.getNoDictNoSortDimIdx();
-    this.measureIdx = tableFieldStat.getMeasureIdx();
-    this.dataTypes = tableFieldStat.getMeasureDataType();
-  }
-
-  /**
-   * constructor
-   * @param sortParameters sort parameters
-   */
-  public SortStepRowHandler(SortParameters sortParameters) {
-    this(new TableFieldStat(sortParameters));
-  }
-
-  /**
-   * Convert carbon row from raw format to 3-parted format.
-   * This method is used in global-sort.
-   *
-   * @param row raw row whose length is the same as field number
-   * @return 3-parted row whose length is 3. (1 for dict dims ,1 for non-dict and complex,
-   * 1 for measures)
-   */
-  public Object[] convertRawRowTo3Parts(Object[] row) {
-    Object[] holder = new Object[3];
-    try {
-      int[] dictDims
-          = new int[this.dictSortDimCnt + this.dictNoSortDimCnt];
-      byte[][] nonDictArray = new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt][];
-      Object[] measures = new Object[this.measureCnt];
-
-      // convert dict & data
-      int idxAcc = 0;
-      for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
-        dictDims[idxAcc++] = (int) row[this.dictSortDimIdx[idx]];
-      }
-
-      // convert dict & no-sort
-      for (int idx = 0; idx < this.dictNoSortDimCnt; idx++) {
-        dictDims[idxAcc++] = (int) row[this.dictNoSortDimIdx[idx]];
-      }
-      // convert no-dict & sort
-      idxAcc = 0;
-      for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
-        nonDictArray[idxAcc++] = (byte[]) row[this.noDictSortDimIdx[idx]];
-      }
-      // convert no-dict & no-sort
-      for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) {
-        nonDictArray[idxAcc++] = (byte[]) row[this.noDictNoSortDimIdx[idx]];
-      }
-
-      // convert measure data
-      for (int idx = 0; idx < this.measureCnt; idx++) {
-        measures[idx] = row[this.measureIdx[idx]];
-      }
-
-      NonDictionaryUtil.prepareOutObj(holder, dictDims, nonDictArray, measures);
-    } catch (Exception e) {
-      throw new RuntimeException("Problem while converting row to 3 parts", e);
-    }
-    return holder;
-  }
-
-  /**
-   * Convert intermediate sort temp row to 3-parted row.
-   * This method is used in the final merge sort to feed rows to the next write step.
-   *
-   * @param sortTempRow intermediate sort temp row
-   * @return 3-parted row
-   */
-  public Object[] convertIntermediateSortTempRowTo3Parted(IntermediateSortTempRow sortTempRow) {
-    int[] dictDims
-        = new int[this.dictSortDimCnt + this.dictNoSortDimCnt];
-    byte[][] noDictArray
-        = new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt][];
-
-    int[] dictNoSortDims = new int[this.dictNoSortDimCnt];
-    byte[][] noDictNoSortDims = new byte[this.noDictNoSortDimCnt][];
-    Object[] measures = new Object[this.measureCnt];
-
-    sortTempRow.unpackNoSortFromBytes(dictNoSortDims, noDictNoSortDims, measures, this.dataTypes);
-
-    // dict dims
-    System.arraycopy(sortTempRow.getDictSortDims(), 0 , dictDims,
-        0, this.dictSortDimCnt);
-    System.arraycopy(dictNoSortDims, 0, dictDims,
-        this.dictSortDimCnt, this.dictNoSortDimCnt);;
-
-    // no dict dims, including complex
-    System.arraycopy(sortTempRow.getNoDictSortDims(), 0,
-        noDictArray, 0, this.noDictSortDimCnt);
-    System.arraycopy(noDictNoSortDims, 0, noDictArray,
-        this.noDictSortDimCnt, this.noDictNoSortDimCnt);
-
-    // measures are already here
-
-    Object[] holder = new Object[3];
-    NonDictionaryUtil.prepareOutObj(holder, dictDims, noDictArray, measures);
-    return holder;
-  }
-
-  /**
-   * Read intermediate sort temp row from InputStream.
-   * This method is used during the merge sort phase to read row from sort temp file.
-   *
-   * @param inputStream input stream
-   * @return a row that contains three parts
-   * @throws IOException if error occrus while reading from stream
-   */
-  public IntermediateSortTempRow readIntermediateSortTempRowFromInputStream(
-      DataInputStream inputStream) throws IOException {
-    int[] dictSortDims = new int[this.dictSortDimCnt];
-    byte[][] noDictSortDims = new byte[this.noDictSortDimCnt][];
-
-    // read dict & sort dim data
-    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
-      dictSortDims[idx] = inputStream.readInt();
-    }
-
-    // read no-dict & sort data
-    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
-      short len = inputStream.readShort();
-      byte[] bytes = new byte[len];
-      inputStream.readFully(bytes);
-      noDictSortDims[idx] = bytes;
-    }
-
-    // read no-dict dims & measures
-    int len = inputStream.readInt();
-    byte[] noSortDimsAndMeasures = new byte[len];
-    inputStream.readFully(noSortDimsAndMeasures);
-
-    return new IntermediateSortTempRow(dictSortDims, noDictSortDims, noSortDimsAndMeasures);
-  }
-
-  /**
-   * Write intermediate sort temp row to OutputStream
-   * This method is used during the merge sort phase to write row to sort temp file.
-   *
-   * @param sortTempRow intermediate sort temp row
-   * @param outputStream output stream
-   * @throws IOException if error occurs while writing to stream
-   */
-  public void writeIntermediateSortTempRowToOutputStream(IntermediateSortTempRow sortTempRow,
-      DataOutputStream outputStream) throws IOException {
-    // write dict & sort dim
-    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
-      outputStream.writeInt(sortTempRow.getDictSortDims()[idx]);
-    }
-
-    // write no-dict & sort dim
-    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
-      byte[] bytes = sortTempRow.getNoDictSortDims()[idx];
-      outputStream.writeShort(bytes.length);
-      outputStream.write(bytes);
-    }
-
-    // write packed no-sort dim & measure
-    outputStream.writeInt(sortTempRow.getNoSortDimsAndMeasures().length);
-    outputStream.write(sortTempRow.getNoSortDimsAndMeasures());
-  }
-
-  /**
-   * Write raw row as an intermediate sort temp row to sort temp file.
-   * This method is used in the beginning of the sort phase. Comparing with converting raw row to
-   * intermediate sort temp row and then writing the converted one, Writing raw row directly will
-   * save the intermediate trivial loss.
-   * This method use an array backend buffer to save memory allocation. The buffer will be reused
-   * for all rows (per thread).
-   *
-   * @param row raw row
-   * @param outputStream output stream
-   * @param rowBuffer array backend buffer
-   * @throws IOException if error occurs while writing to stream
-   */
-  public void writeRawRowAsIntermediateSortTempRowToOutputStream(Object[] row,
-      DataOutputStream outputStream, ByteBuffer rowBuffer) throws IOException {
-    // write dict & sort
-    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
-      outputStream.writeInt((int) row[this.dictSortDimIdx[idx]]);
-    }
-
-    // write no-dict & sort
-    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
-      byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]];
-      outputStream.writeShort(bytes.length);
-      outputStream.write(bytes);
-    }
-
-    // pack no-sort
-    rowBuffer.clear();
-    packNoSortFieldsToBytes(row, rowBuffer);
-    rowBuffer.flip();
-    int packSize = rowBuffer.limit();
-
-    // write no-sort
-    outputStream.writeInt(packSize);
-    outputStream.write(rowBuffer.array(), 0, packSize);
-  }
-
-  /**
-   * Read intermediate sort temp row from unsafe memory.
-   * This method is used during merge sort phase for off-heap sort.
-   *
-   * @param baseObject base object of memory block
-   * @param address address of the row
-   * @return intermediate sort temp row
-   */
-  public IntermediateSortTempRow readIntermediateSortTempRowFromUnsafeMemory(Object baseObject,
-      long address) {
-    int size = 0;
-
-    int[] dictSortDims = new int[this.dictSortDimCnt];
-    byte[][] noDictSortDims = new byte[this.noDictSortDimCnt][];
-
-    // read dict & sort dim
-    for (int idx = 0; idx < dictSortDims.length; idx++) {
-      dictSortDims[idx] = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-      size += 4;
-    }
-
-    // read no-dict & sort dim
-    for (int idx = 0; idx < noDictSortDims.length; idx++) {
-      short length = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-      size += 2;
-      byte[] bytes = new byte[length];
-      CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
-          bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length);
-      size += length;
-      noDictSortDims[idx] = bytes;
-    }
-
-    // read no-sort dims & measures
-    int len = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-    size += 4;
-    byte[] noSortDimsAndMeasures = new byte[len];
-    CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
-        noSortDimsAndMeasures, CarbonUnsafe.BYTE_ARRAY_OFFSET, len);
-
-    return new IntermediateSortTempRow(dictSortDims, noDictSortDims, noSortDimsAndMeasures);
-  }
-
-  /**
-   * Write intermediate sort temp row directly from unsafe memory to stream.
-   * This method is used at the late beginning of the sort phase to write in-memory pages
-   * to sort temp file. Comparing with reading intermediate sort temp row from memory and then
-   * writing it, Writing directly from memory to stream will save the intermediate trivial loss.
-   *
-   * @param baseObject base object of the memory block
-   * @param address base address of the row
-   * @param outputStream output stream
-   * @throws IOException if error occurs while writing to stream
-   */
-  public void writeIntermediateSortTempRowFromUnsafeMemoryToStream(Object baseObject,
-      long address, DataOutputStream outputStream) throws IOException {
-    int size = 0;
-
-    // dict & sort
-    for (int idx = 0; idx < dictSortDimCnt; idx++) {
-      outputStream.writeInt(CarbonUnsafe.getUnsafe().getInt(baseObject, address + size));
-      size += 4;
-    }
-
-    // no-dict & sort
-    for (int idx = 0; idx < noDictSortDimCnt; idx++) {
-      short length = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-      size += 2;
-      byte[] bytes = new byte[length];
-      CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
-          bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length);
-      size += length;
-
-      outputStream.writeShort(length);
-      outputStream.write(bytes);
-    }
-
-    // packed no-sort & measure
-    int len = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-    size += 4;
-    byte[] noSortDimsAndMeasures = new byte[len];
-    CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
-        noSortDimsAndMeasures, CarbonUnsafe.BYTE_ARRAY_OFFSET, len);
-    size += len;
-
-    outputStream.writeInt(len);
-    outputStream.write(noSortDimsAndMeasures);
-  }
-
-  /**
-   * Write raw row as an intermediate sort temp row to memory.
-   * This method is used in the beginning of the off-heap sort phase. Comparing with converting
-   * raw row to intermediate sort temp row and then writing the converted one,
-   * Writing raw row directly will save the intermediate trivial loss.
-   * This method use an array backend buffer to save memory allocation. The buffer will be reused
-   * for all rows (per thread).
-   *
-   * @param row raw row
-   * @param baseObject base object of the memory block
-   * @param address base address for the row
-   * @param rowBuffer array backend buffer
-   * @return number of bytes written to memory
-   */
-  public int writeRawRowAsIntermediateSortTempRowToUnsafeMemory(Object[] row,
-      Object baseObject, long address, ByteBuffer rowBuffer) {
-    int size = 0;
-    // write dict & sort
-    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
-      CarbonUnsafe.getUnsafe()
-          .putInt(baseObject, address + size, (int) row[this.dictSortDimIdx[idx]]);
-      size += 4;
-    }
-
-    // write no-dict & sort
-    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
-      byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]];
-      CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) bytes.length);
-      size += 2;
-      CarbonUnsafe.getUnsafe()
-          .copyMemory(bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, address + size,
-              bytes.length);
-      size += bytes.length;
-    }
-
-    // convert pack no-sort
-    rowBuffer.clear();
-    packNoSortFieldsToBytes(row, rowBuffer);
-    rowBuffer.flip();
-    int packSize = rowBuffer.limit();
-
-    // write no-sort
-    CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, packSize);
-    size += 4;
-    CarbonUnsafe.getUnsafe()
-        .copyMemory(rowBuffer.array(), CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, address + size,
-            packSize);
-    size += packSize;
-    return size;
-  }
-
-  /**
-   * Pack to no-sort fields to byte array
-   *
-   * @param row raw row
-   * @param rowBuffer byte array backend buffer
-   */
-  private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) {
-    // convert dict & no-sort
-    for (int idx = 0; idx < this.dictNoSortDimCnt; idx++) {
-      rowBuffer.putInt((int) row[this.dictNoSortDimIdx[idx]]);
-    }
-    // convert no-dict & no-sort
-    for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) {
-      byte[] bytes = (byte[]) row[this.noDictNoSortDimIdx[idx]];
-      rowBuffer.putShort((short) bytes.length);
-      rowBuffer.put(bytes);
-    }
-
-    // convert measure
-    Object tmpValue;
-    DataType tmpDataType;
-    for (int idx = 0; idx < this.measureCnt; idx++) {
-      tmpValue = row[this.measureIdx[idx]];
-      tmpDataType = this.dataTypes[idx];
-      if (null == tmpValue) {
-        rowBuffer.put((byte) 0);
-        continue;
-      }
-      rowBuffer.put((byte) 1);
-      if (DataTypes.BOOLEAN == tmpDataType) {
-        if ((boolean) tmpValue) {
-          rowBuffer.put((byte) 1);
-        } else {
-          rowBuffer.put((byte) 0);
-        }
-      } else if (DataTypes.SHORT == tmpDataType) {
-        rowBuffer.putShort((Short) tmpValue);
-      } else if (DataTypes.INT == tmpDataType) {
-        rowBuffer.putInt((Integer) tmpValue);
-      } else if (DataTypes.LONG == tmpDataType) {
-        rowBuffer.putLong((Long) tmpValue);
-      } else if (DataTypes.DOUBLE == tmpDataType) {
-        rowBuffer.putDouble((Double) tmpValue);
-      } else if (DataTypes.isDecimal(tmpDataType)) {
-        byte[] decimalBytes = DataTypeUtil.bigDecimalToByte((BigDecimal) tmpValue);
-        rowBuffer.putShort((short) decimalBytes.length);
-        rowBuffer.put(decimalBytes);
-      } else {
-        throw new IllegalArgumentException("Unsupported data type: " + tmpDataType);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
new file mode 100644
index 0000000..c4e4756
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort;
+
+import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+
+public class SortStepRowUtil {
+  private int measureCount;
+  private int dimensionCount;
+  private int complexDimensionCount;
+  private int noDictionaryCount;
+  private int[] dictDimIdx;
+  private int[] nonDictIdx;
+  private int[] measureIdx;
+
+  public SortStepRowUtil(SortParameters parameters) {
+    this.measureCount = parameters.getMeasureColCount();
+    this.dimensionCount = parameters.getDimColCount();
+    this.complexDimensionCount = parameters.getComplexDimColCount();
+    this.noDictionaryCount = parameters.getNoDictionaryCount();
+    boolean[] isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
+
+    int index = 0;
+    int nonDicIndex = 0;
+    int allCount = 0;
+
+    // be careful that the default value is 0
+    this.dictDimIdx = new int[dimensionCount - noDictionaryCount];
+    this.nonDictIdx = new int[noDictionaryCount + complexDimensionCount];
+    this.measureIdx = new int[measureCount];
+
+    // indices for dict dim columns
+    for (int i = 0; i < isNoDictionaryDimensionColumn.length; i++) {
+      if (isNoDictionaryDimensionColumn[i]) {
+        nonDictIdx[nonDicIndex++] = i;
+      } else {
+        dictDimIdx[index++] = allCount;
+      }
+      allCount++;
+    }
+
+    // indices for non dict dim/complex columns
+    for (int i = 0; i < complexDimensionCount; i++) {
+      nonDictIdx[nonDicIndex++] = allCount;
+      allCount++;
+    }
+
+    // indices for measure columns
+    for (int i = 0; i < measureCount; i++) {
+      measureIdx[i] = allCount;
+      allCount++;
+    }
+  }
+
+  public Object[] convertRow(Object[] data) {
+    // create new row of size 3 (1 for dims , 1 for high card , 1 for measures)
+    Object[] holder = new Object[3];
+    try {
+
+      int[] dictDims = new int[dimensionCount - noDictionaryCount];
+      byte[][] nonDictArray = new byte[noDictionaryCount + complexDimensionCount][];
+      Object[] measures = new Object[measureCount];
+
+      // write dict dim data
+      for (int idx = 0; idx < dictDimIdx.length; idx++) {
+        dictDims[idx] = (int) data[dictDimIdx[idx]];
+      }
+
+      // write non dict dim data
+      for (int idx = 0; idx < nonDictIdx.length; idx++) {
+        nonDictArray[idx] = (byte[]) data[nonDictIdx[idx]];
+      }
+
+      // write measure data
+      for (int idx = 0; idx < measureIdx.length; idx++) {
+        measures[idx] = data[measureIdx[idx]];
+      }
+      NonDictionaryUtil.prepareOutObj(holder, dictDims, nonDictArray, measures);
+
+      // increment number if record read
+    } catch (Exception e) {
+      throw new RuntimeException("Problem while converting row ", e);
+    }
+    //return out row
+    return holder;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
index 7ea5cb3..e5583c2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
@@ -19,20 +19,35 @@ package org.apache.carbondata.processing.loading.sort.unsafe;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
+import java.math.BigDecimal;
+import java.util.Arrays;
 
+import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.memory.IntPointerBuffer;
 import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.memory.UnsafeSortMemoryManager;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
-import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.DataTypeUtil;
 
 /**
  * It can keep the data of prescribed size data in offheap/onheap memory and returns it when needed
  */
 public class UnsafeCarbonRowPage {
+
+  private boolean[] noDictionaryDimensionMapping;
+
+  private boolean[] noDictionarySortColumnMapping;
+
+  private int dimensionSize;
+
+  private int measureSize;
+
+  private DataType[] measureDataType;
+
+  private long[] nullSetWords;
+
   private IntPointerBuffer buffer;
 
   private int lastSize;
@@ -47,14 +62,16 @@ public class UnsafeCarbonRowPage {
 
   private long taskId;
 
-  private TableFieldStat tableFieldStat;
-  private SortStepRowHandler sortStepRowHandler;
-
-  public UnsafeCarbonRowPage(TableFieldStat tableFieldStat, MemoryBlock memoryBlock,
-      boolean saveToDisk, long taskId) {
-    this.tableFieldStat = tableFieldStat;
-    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
+  public UnsafeCarbonRowPage(boolean[] noDictionaryDimensionMapping,
+      boolean[] noDictionarySortColumnMapping, int dimensionSize, int measureSize, DataType[] type,
+      MemoryBlock memoryBlock, boolean saveToDisk, long taskId) {
+    this.noDictionaryDimensionMapping = noDictionaryDimensionMapping;
+    this.noDictionarySortColumnMapping = noDictionarySortColumnMapping;
+    this.dimensionSize = dimensionSize;
+    this.measureSize = measureSize;
+    this.measureDataType = type;
     this.saveToDisk = saveToDisk;
+    this.nullSetWords = new long[((measureSize - 1) >> 6) + 1];
     this.taskId = taskId;
     buffer = new IntPointerBuffer(this.taskId);
     this.dataBlock = memoryBlock;
@@ -63,44 +80,255 @@ public class UnsafeCarbonRowPage {
     this.managerType = MemoryManagerType.UNSAFE_MEMORY_MANAGER;
   }
 
-  public int addRow(Object[] row, ByteBuffer rowBuffer) {
-    int size = addRow(row, dataBlock.getBaseOffset() + lastSize, rowBuffer);
+  public int addRow(Object[] row) {
+    int size = addRow(row, dataBlock.getBaseOffset() + lastSize);
     buffer.set(lastSize);
     lastSize = lastSize + size;
     return size;
   }
 
-  /**
-   * add raw row as intermidiate sort temp row to page
-   *
-   * @param row
-   * @param address
-   * @return
-   */
-  private int addRow(Object[] row, long address, ByteBuffer rowBuffer) {
-    return sortStepRowHandler.writeRawRowAsIntermediateSortTempRowToUnsafeMemory(row,
-        dataBlock.getBaseObject(), address, rowBuffer);
+  private int addRow(Object[] row, long address) {
+    if (row == null) {
+      throw new RuntimeException("Row is null ??");
+    }
+    int dimCount = 0;
+    int size = 0;
+    Object baseObject = dataBlock.getBaseObject();
+    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
+      if (noDictionaryDimensionMapping[dimCount]) {
+        byte[] col = (byte[]) row[dimCount];
+        CarbonUnsafe.getUnsafe()
+            .putShort(baseObject, address + size, (short) col.length);
+        size += 2;
+        CarbonUnsafe.getUnsafe().copyMemory(col, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
+            address + size, col.length);
+        size += col.length;
+      } else {
+        int value = (int) row[dimCount];
+        CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, value);
+        size += 4;
+      }
+    }
+
+    // write complex dimensions here.
+    for (; dimCount < dimensionSize; dimCount++) {
+      byte[] col = (byte[]) row[dimCount];
+      CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) col.length);
+      size += 2;
+      CarbonUnsafe.getUnsafe().copyMemory(col, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
+          address + size, col.length);
+      size += col.length;
+    }
+    Arrays.fill(nullSetWords, 0);
+    int nullSetSize = nullSetWords.length * 8;
+    int nullWordLoc = size;
+    size += nullSetSize;
+    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
+      Object value = row[mesCount + dimensionSize];
+      if (null != value) {
+        DataType dataType = measureDataType[mesCount];
+        if (dataType == DataTypes.BOOLEAN) {
+          Boolean bval = (Boolean) value;
+          CarbonUnsafe.getUnsafe().putBoolean(baseObject, address + size, bval);
+          size += 1;
+        } else if (dataType == DataTypes.SHORT) {
+          Short sval = (Short) value;
+          CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, sval);
+          size += 2;
+        } else if (dataType == DataTypes.INT) {
+          Integer ival = (Integer) value;
+          CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, ival);
+          size += 4;
+        } else if (dataType == DataTypes.LONG) {
+          Long val = (Long) value;
+          CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, val);
+          size += 8;
+        } else if (dataType == DataTypes.DOUBLE) {
+          Double doubleVal = (Double) value;
+          CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, doubleVal);
+          size += 8;
+        } else if (DataTypes.isDecimal(dataType)) {
+          BigDecimal decimalVal = (BigDecimal) value;
+          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(decimalVal);
+          CarbonUnsafe.getUnsafe()
+              .putShort(baseObject, address + size, (short) bigDecimalInBytes.length);
+          size += 2;
+          CarbonUnsafe.getUnsafe()
+              .copyMemory(bigDecimalInBytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
+                  address + size, bigDecimalInBytes.length);
+          size += bigDecimalInBytes.length;
+        } else {
+          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
+        }
+        set(nullSetWords, mesCount);
+      } else {
+        unset(nullSetWords, mesCount);
+      }
+    }
+    CarbonUnsafe.getUnsafe().copyMemory(nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET, baseObject,
+        address + nullWordLoc, nullSetSize);
+    return size;
   }
 
-  /**
-   * get one row from memory address
-   * @param address address
-   * @return one row
-   */
-  public IntermediateSortTempRow getRow(long address) {
-    return sortStepRowHandler.readIntermediateSortTempRowFromUnsafeMemory(
-        dataBlock.getBaseObject(), address);
+  public Object[] getRow(long address, Object[] rowToFill) {
+    int dimCount = 0;
+    int size = 0;
+
+    Object baseObject = dataBlock.getBaseObject();
+    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
+      if (noDictionaryDimensionMapping[dimCount]) {
+        short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+        byte[] col = new byte[aShort];
+        size += 2;
+        CarbonUnsafe.getUnsafe()
+            .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET,
+                col.length);
+        size += col.length;
+        rowToFill[dimCount] = col;
+      } else {
+        int anInt = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+        size += 4;
+        rowToFill[dimCount] = anInt;
+      }
+    }
+
+    // write complex dimensions here.
+    for (; dimCount < dimensionSize; dimCount++) {
+      short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+      byte[] col = new byte[aShort];
+      size += 2;
+      CarbonUnsafe.getUnsafe()
+          .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET, col.length);
+      size += col.length;
+      rowToFill[dimCount] = col;
+    }
+
+    int nullSetSize = nullSetWords.length * 8;
+    Arrays.fill(nullSetWords, 0);
+    CarbonUnsafe.getUnsafe()
+        .copyMemory(baseObject, address + size, nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET,
+            nullSetSize);
+    size += nullSetSize;
+
+    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
+      if (isSet(nullSetWords, mesCount)) {
+        DataType dataType = measureDataType[mesCount];
+        if (dataType == DataTypes.BOOLEAN) {
+          Boolean bval = CarbonUnsafe.getUnsafe().getBoolean(baseObject, address + size);
+          size += 1;
+          rowToFill[dimensionSize + mesCount] = bval;
+        } else if (dataType == DataTypes.SHORT) {
+          Short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+          size += 2;
+          rowToFill[dimensionSize + mesCount] = sval;
+        } else if (dataType == DataTypes.INT) {
+          Integer ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+          size += 4;
+          rowToFill[dimensionSize + mesCount] = ival;
+        } else if (dataType == DataTypes.LONG) {
+          Long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
+          size += 8;
+          rowToFill[dimensionSize + mesCount] = val;
+        } else if (dataType == DataTypes.DOUBLE) {
+          Double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
+          size += 8;
+          rowToFill[dimensionSize + mesCount] = doubleVal;
+        } else if (DataTypes.isDecimal(dataType)) {
+          short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+          byte[] bigDecimalInBytes = new byte[aShort];
+          size += 2;
+          CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
+              CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
+          size += bigDecimalInBytes.length;
+          rowToFill[dimensionSize + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
+        } else {
+          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
+        }
+      } else {
+        rowToFill[dimensionSize + mesCount] = null;
+      }
+    }
+    return rowToFill;
   }
 
-  /**
-   * write a row to stream
-   * @param address address of a row
-   * @param stream stream
-   * @throws IOException
-   */
-  public void writeRow(long address, DataOutputStream stream) throws IOException {
-    sortStepRowHandler.writeIntermediateSortTempRowFromUnsafeMemoryToStream(
-        dataBlock.getBaseObject(), address, stream);
+  public void fillRow(long address, DataOutputStream stream) throws IOException {
+    int dimCount = 0;
+    int size = 0;
+
+    Object baseObject = dataBlock.getBaseObject();
+    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
+      if (noDictionaryDimensionMapping[dimCount]) {
+        short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+        byte[] col = new byte[aShort];
+        size += 2;
+        CarbonUnsafe.getUnsafe()
+            .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET,
+                col.length);
+        size += col.length;
+        stream.writeShort(aShort);
+        stream.write(col);
+      } else {
+        int anInt = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+        size += 4;
+        stream.writeInt(anInt);
+      }
+    }
+
+    // write complex dimensions here.
+    for (; dimCount < dimensionSize; dimCount++) {
+      short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+      byte[] col = new byte[aShort];
+      size += 2;
+      CarbonUnsafe.getUnsafe()
+          .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET, col.length);
+      size += col.length;
+      stream.writeShort(aShort);
+      stream.write(col);
+    }
+
+    int nullSetSize = nullSetWords.length * 8;
+    Arrays.fill(nullSetWords, 0);
+    CarbonUnsafe.getUnsafe()
+        .copyMemory(baseObject, address + size, nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET,
+            nullSetSize);
+    size += nullSetSize;
+    for (int i = 0; i < nullSetWords.length; i++) {
+      stream.writeLong(nullSetWords[i]);
+    }
+
+    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
+      if (isSet(nullSetWords, mesCount)) {
+        DataType dataType = measureDataType[mesCount];
+        if (dataType == DataTypes.SHORT) {
+          short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+          size += 2;
+          stream.writeShort(sval);
+        } else if (dataType == DataTypes.INT) {
+          int ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+          size += 4;
+          stream.writeInt(ival);
+        } else if (dataType == DataTypes.LONG) {
+          long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
+          size += 8;
+          stream.writeLong(val);
+        } else if (dataType == DataTypes.DOUBLE) {
+          double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
+          size += 8;
+          stream.writeDouble(doubleVal);
+        } else if (DataTypes.isDecimal(dataType)) {
+          short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+          byte[] bigDecimalInBytes = new byte[aShort];
+          size += 2;
+          CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
+              CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
+          size += bigDecimalInBytes.length;
+          stream.writeShort(aShort);
+          stream.write(bigDecimalInBytes);
+        } else {
+          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
+        }
+      }
+    }
   }
 
   public void freeMemory() {
@@ -134,8 +362,27 @@ public class UnsafeCarbonRowPage {
     return dataBlock;
   }
 
-  public TableFieldStat getTableFieldStat() {
-    return tableFieldStat;
+  public static void set(long[] words, int index) {
+    int wordOffset = (index >> 6);
+    words[wordOffset] |= (1L << index);
+  }
+
+  public static void unset(long[] words, int index) {
+    int wordOffset = (index >> 6);
+    words[wordOffset] &= ~(1L << index);
+  }
+
+  public static boolean isSet(long[] words, int index) {
+    int wordOffset = (index >> 6);
+    return ((words[wordOffset] & (1L << index)) != 0);
+  }
+
+  public boolean[] getNoDictionaryDimensionMapping() {
+    return noDictionaryDimensionMapping;
+  }
+
+  public boolean[] getNoDictionarySortColumnMapping() {
+    return noDictionarySortColumnMapping;
   }
 
   public void setNewDataBlock(MemoryBlock newMemoryBlock) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
index 5d038d3..4dd5e44 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
@@ -20,7 +20,6 @@ package org.apache.carbondata.processing.loading.sort.unsafe;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -42,14 +41,13 @@ import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparator;
-import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparatorForNormalDims;
+import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparatorForNormalDIms;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeIntermediateMerger;
 import org.apache.carbondata.processing.loading.sort.unsafe.sort.TimSort;
 import org.apache.carbondata.processing.loading.sort.unsafe.sort.UnsafeIntSortDataFormat;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 public class UnsafeSortDataRows {
@@ -71,8 +69,7 @@ public class UnsafeSortDataRows {
    */
 
   private SortParameters parameters;
-  private TableFieldStat tableFieldStat;
-  private ThreadLocal<ByteBuffer> rowBuffer;
+
   private UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger;
 
   private UnsafeCarbonRowPage rowPage;
@@ -97,13 +94,7 @@ public class UnsafeSortDataRows {
   public UnsafeSortDataRows(SortParameters parameters,
       UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger, int inMemoryChunkSize) {
     this.parameters = parameters;
-    this.tableFieldStat = new TableFieldStat(parameters);
-    this.rowBuffer = new ThreadLocal<ByteBuffer>() {
-      @Override protected ByteBuffer initialValue() {
-        byte[] backedArray = new byte[2 * 1024 * 1024];
-        return ByteBuffer.wrap(backedArray);
-      }
-    };
+
     this.unsafeInMemoryIntermediateFileMerger = unsafeInMemoryIntermediateFileMerger;
 
     // observer of writing file in thread
@@ -136,7 +127,11 @@ public class UnsafeSortDataRows {
     if (isMemoryAvailable) {
       UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(baseBlock.size());
     }
-    this.rowPage = new UnsafeCarbonRowPage(tableFieldStat, baseBlock, !isMemoryAvailable, taskId);
+    this.rowPage = new UnsafeCarbonRowPage(parameters.getNoDictionaryDimnesionColumn(),
+        parameters.getNoDictionarySortColumn(),
+        parameters.getDimColCount() + parameters.getComplexDimColCount(),
+        parameters.getMeasureColCount(), parameters.getMeasureDataType(), baseBlock,
+        !isMemoryAvailable, taskId);
     // Delete if any older file exists in sort temp folder
     deleteSortLocationIfExists();
 
@@ -183,7 +178,7 @@ public class UnsafeSortDataRows {
   private void addBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
     for (int i = 0; i < size; i++) {
       if (rowPage.canAdd()) {
-        bytesAdded += rowPage.addRow(rowBatch[i], rowBuffer.get());
+        bytesAdded += rowPage.addRow(rowBatch[i]);
       } else {
         try {
           if (enableInMemoryIntermediateMerge) {
@@ -199,8 +194,15 @@ public class UnsafeSortDataRows {
           if (!saveToDisk) {
             UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(memoryBlock.size());
           }
-          rowPage = new UnsafeCarbonRowPage(tableFieldStat, memoryBlock, saveToDisk, taskId);
-          bytesAdded += rowPage.addRow(rowBatch[i], rowBuffer.get());
+          rowPage = new UnsafeCarbonRowPage(
+                  parameters.getNoDictionaryDimnesionColumn(),
+                  parameters.getNoDictionarySortColumn(),
+                  parameters.getDimColCount() + parameters.getComplexDimColCount(),
+                  parameters.getMeasureColCount(),
+                  parameters.getMeasureDataType(),
+                  memoryBlock,
+                  saveToDisk, taskId);
+          bytesAdded += rowPage.addRow(rowBatch[i]);
         } catch (Exception e) {
           LOGGER.error(
                   "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
@@ -218,7 +220,7 @@ public class UnsafeSortDataRows {
     // if record holder list size is equal to sort buffer size then it will
     // sort the list and then write current list data to file
     if (rowPage.canAdd()) {
-      rowPage.addRow(row, rowBuffer.get());
+      rowPage.addRow(row);
     } else {
       try {
         if (enableInMemoryIntermediateMerge) {
@@ -233,8 +235,13 @@ public class UnsafeSortDataRows {
         if (!saveToDisk) {
           UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(memoryBlock.size());
         }
-        rowPage = new UnsafeCarbonRowPage(tableFieldStat, memoryBlock, saveToDisk, taskId);
-        rowPage.addRow(row, rowBuffer.get());
+        rowPage = new UnsafeCarbonRowPage(
+            parameters.getNoDictionaryDimnesionColumn(),
+            parameters.getNoDictionarySortColumn(),
+            parameters.getDimColCount(), parameters.getMeasureColCount(),
+            parameters.getMeasureDataType(), memoryBlock,
+            saveToDisk, taskId);
+        rowPage.addRow(row);
       } catch (Exception e) {
         LOGGER.error(
             "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
@@ -262,7 +269,7 @@ public class UnsafeSortDataRows {
             new UnsafeRowComparator(rowPage));
       } else {
         timSort.sort(rowPage.getBuffer(), 0, rowPage.getBuffer().getActualSize(),
-            new UnsafeRowComparatorForNormalDims(rowPage));
+            new UnsafeRowComparatorForNormalDIms(rowPage));
       }
       unsafeInMemoryIntermediateFileMerger.addDataChunkToMerge(rowPage);
     } else {
@@ -288,9 +295,10 @@ public class UnsafeSortDataRows {
       // write number of entries to the file
       stream.writeInt(actualSize);
       for (int i = 0; i < actualSize; i++) {
-        rowPage.writeRow(
-            rowPage.getBuffer().get(i) + rowPage.getDataBlock().getBaseOffset(), stream);
+        rowPage.fillRow(rowPage.getBuffer().get(i) + rowPage.getDataBlock().getBaseOffset(),
+            stream);
       }
+
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
     } finally {
@@ -359,7 +367,7 @@ public class UnsafeSortDataRows {
               new UnsafeRowComparator(page));
         } else {
           timSort.sort(page.getBuffer(), 0, page.getBuffer().getActualSize(),
-              new UnsafeRowComparatorForNormalDims(page));
+              new UnsafeRowComparatorForNormalDIms(page));
         }
         if (page.isSaveToDisk()) {
           // create a new file every time
@@ -372,8 +380,7 @@ public class UnsafeSortDataRows {
           writeDataToFile(page, sortTempFile);
           LOGGER.info("Time taken to sort row page with size" + page.getBuffer().getActualSize()
               + " and write is: " + (System.currentTimeMillis() - startTime) + ": location:"
-              + sortTempFile + ", sort temp file size in MB is "
-              + sortTempFile.length() * 0.1 * 10 / 1024 / 1024);
+              + sortTempFile);
           page.freeMemory();
           // add sort temp filename to and arrayList. When the list size reaches 20 then
           // intermediate merging of sort temp files will be triggered

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
index 33342dc..d02be9b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
@@ -23,25 +23,63 @@ import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
-import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeRowComparator implements Comparator<UnsafeCarbonRow> {
+
+  /**
+   * mapping of dictionary and no dictionary of sort_columns.
+   */
+  private boolean[] noDictionarySortColumnMaping;
+
   private Object baseObject;
-  private TableFieldStat tableFieldStat;
-  private int dictSizeInMemory;
 
   public UnsafeRowComparator(UnsafeCarbonRowPage rowPage) {
+    this.noDictionarySortColumnMaping = rowPage.getNoDictionarySortColumnMapping();
     this.baseObject = rowPage.getDataBlock().getBaseObject();
-    this.tableFieldStat = rowPage.getTableFieldStat();
-    this.dictSizeInMemory = (tableFieldStat.getDictSortDimCnt()
-        + tableFieldStat.getDictNoSortDimCnt()) * 4;
   }
 
   /**
    * Below method will be used to compare two mdkey
    */
   public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
-    return compare(rowL, baseObject, rowR, baseObject);
+    int diff = 0;
+    long rowA = rowL.address;
+    long rowB = rowR.address;
+    int sizeA = 0;
+    int sizeB = 0;
+    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
+      if (isNoDictionary) {
+        short aShort1 = CarbonUnsafe.getUnsafe().getShort(baseObject, rowA + sizeA);
+        byte[] byteArr1 = new byte[aShort1];
+        sizeA += 2;
+        CarbonUnsafe.getUnsafe().copyMemory(baseObject, rowA + sizeA, byteArr1,
+            CarbonUnsafe.BYTE_ARRAY_OFFSET, aShort1);
+        sizeA += aShort1;
+
+        short aShort2 = CarbonUnsafe.getUnsafe().getShort(baseObject, rowB + sizeB);
+        byte[] byteArr2 = new byte[aShort2];
+        sizeB += 2;
+        CarbonUnsafe.getUnsafe().copyMemory(baseObject, rowB + sizeB, byteArr2,
+            CarbonUnsafe.BYTE_ARRAY_OFFSET, aShort2);
+        sizeB += aShort2;
+
+        int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
+        if (difference != 0) {
+          return difference;
+        }
+      } else {
+        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
+        sizeA += 4;
+        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
+        sizeB += 4;
+        diff = dimFieldA - dimFieldB;
+        if (diff != 0) {
+          return diff;
+        }
+      }
+    }
+
+    return diff;
   }
 
   /**
@@ -52,40 +90,35 @@ public class UnsafeRowComparator implements Comparator<UnsafeCarbonRow> {
     int diff = 0;
     long rowA = rowL.address;
     long rowB = rowR.address;
-    int sizeInDictPartA = 0;
-
-    int sizeInNonDictPartA = 0;
-    int sizeInDictPartB = 0;
-    int sizeInNonDictPartB = 0;
-    for (boolean isNoDictionary : tableFieldStat.getIsSortColNoDictFlags()) {
+    int sizeA = 0;
+    int sizeB = 0;
+    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
       if (isNoDictionary) {
-        short lengthA = CarbonUnsafe.getUnsafe().getShort(baseObjectL,
-            rowA + dictSizeInMemory + sizeInNonDictPartA);
-        byte[] byteArr1 = new byte[lengthA];
-        sizeInNonDictPartA += 2;
+        short aShort1 = CarbonUnsafe.getUnsafe().getShort(baseObjectL, rowA + sizeA);
+        byte[] byteArr1 = new byte[aShort1];
+        sizeA += 2;
         CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObjectL, rowA + dictSizeInMemory + sizeInNonDictPartA,
-                byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthA);
-        sizeInNonDictPartA += lengthA;
+            .copyMemory(baseObjectL, rowA + sizeA, byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET,
+                aShort1);
+        sizeA += aShort1;
 
-        short lengthB = CarbonUnsafe.getUnsafe().getShort(baseObjectR,
-            rowB + dictSizeInMemory + sizeInNonDictPartB);
-        byte[] byteArr2 = new byte[lengthB];
-        sizeInNonDictPartB += 2;
+        short aShort2 = CarbonUnsafe.getUnsafe().getShort(baseObjectR, rowB + sizeB);
+        byte[] byteArr2 = new byte[aShort2];
+        sizeB += 2;
         CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObjectR, rowB + dictSizeInMemory + sizeInNonDictPartB,
-                byteArr2, CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthB);
-        sizeInNonDictPartB += lengthB;
+            .copyMemory(baseObjectR, rowB + sizeB, byteArr2, CarbonUnsafe.BYTE_ARRAY_OFFSET,
+                aShort2);
+        sizeB += aShort2;
 
         int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
         if (difference != 0) {
           return difference;
         }
       } else {
-        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObjectL, rowA + sizeInDictPartA);
-        sizeInDictPartA += 4;
-        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObjectR, rowB + sizeInDictPartB);
-        sizeInDictPartB += 4;
+        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObjectL, rowA + sizeA);
+        sizeA += 4;
+        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObjectR, rowB + sizeB);
+        sizeB += 4;
         diff = dimFieldA - dimFieldB;
         if (diff != 0) {
           return diff;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
new file mode 100644
index 0000000..483dcb2
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort.unsafe.comparator;
+
+import java.util.Comparator;
+
+import org.apache.carbondata.core.memory.CarbonUnsafe;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
+
+public class UnsafeRowComparatorForNormalDIms implements Comparator<UnsafeCarbonRow> {
+
+  private Object baseObject;
+
+  private int numberOfSortColumns;
+
+  public UnsafeRowComparatorForNormalDIms(UnsafeCarbonRowPage rowPage) {
+    this.baseObject = rowPage.getDataBlock().getBaseObject();
+    this.numberOfSortColumns = rowPage.getNoDictionarySortColumnMapping().length;
+  }
+
+  /**
+   * Below method will be used to compare two mdkey
+   */
+  public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
+    int diff = 0;
+    long rowA = rowL.address;
+    long rowB = rowR.address;
+    int sizeA = 0;
+    int sizeB = 0;
+    for (int i = 0; i < numberOfSortColumns; i++) {
+      int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
+      sizeA += 4;
+      int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
+      sizeB += 4;
+      diff = dimFieldA - dimFieldB;
+      if (diff != 0) {
+        return diff;
+      }
+    }
+
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
deleted file mode 100644
index e9cfb1c..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.sort.unsafe.comparator;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.memory.CarbonUnsafe;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
-
-public class UnsafeRowComparatorForNormalDims implements Comparator<UnsafeCarbonRow> {
-
-  private Object baseObject;
-
-  private int numberOfSortColumns;
-
-  public UnsafeRowComparatorForNormalDims(UnsafeCarbonRowPage rowPage) {
-    this.baseObject = rowPage.getDataBlock().getBaseObject();
-    this.numberOfSortColumns = rowPage.getTableFieldStat().getIsSortColNoDictFlags().length;
-  }
-
-  /**
-   * Below method will be used to compare two mdkey
-   */
-  public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
-    int diff = 0;
-    long rowA = rowL.address;
-    long rowB = rowR.address;
-    int sizeA = 0;
-    int sizeB = 0;
-    for (int i = 0; i < numberOfSortColumns; i++) {
-      int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
-      sizeA += 4;
-      int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
-      sizeB += 4;
-      diff = dimFieldA - dimFieldB;
-      if (diff != 0) {
-        return diff;
-      }
-    }
-
-    return diff;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
index d790c41..686e855 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 /**
@@ -29,7 +28,7 @@ public interface SortTempChunkHolder extends Comparable<SortTempChunkHolder> {
 
   void readRow()  throws CarbonSortKeyAndGroupByException;
 
-  IntermediateSortTempRow getRow();
+  Object[] getRow();
 
   int numberOfRows();
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
index a776db1..6b0cfa6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
@@ -19,10 +19,9 @@ package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeInMemoryIntermediateDataMerger;
-import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
 
 public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
 
@@ -39,18 +38,21 @@ public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
 
   private UnsafeCarbonRowPage[] rowPages;
 
-  private IntermediateSortTempRowComparator comparator;
+  private NewRowComparator comparator;
 
-  private IntermediateSortTempRow currentRow;
+  private Object[] currentRow;
+
+  private int columnSize;
 
   public UnsafeFinalMergePageHolder(UnsafeInMemoryIntermediateDataMerger merger,
-      boolean[] noDictSortColumnMapping) {
+      boolean[] noDictSortColumnMapping, int columnSize) {
     this.actualSize = merger.getEntryCount();
     this.mergedAddresses = merger.getMergedAddresses();
     this.rowPageIndexes = merger.getRowPageIndexes();
     this.rowPages = merger.getUnsafeCarbonRowPages();
     LOGGER.audit("Processing unsafe inmemory rows page with size : " + actualSize);
-    this.comparator = new IntermediateSortTempRowComparator(noDictSortColumnMapping);
+    this.comparator = new NewRowComparator(noDictSortColumnMapping);
+    this.columnSize = columnSize;
   }
 
   public boolean hasNext() {
@@ -61,11 +63,12 @@ public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
   }
 
   public void readRow() {
-    currentRow = rowPages[rowPageIndexes[counter]].getRow(mergedAddresses[counter]);
+    currentRow = new Object[columnSize];
+    rowPages[rowPageIndexes[counter]].getRow(mergedAddresses[counter], currentRow);
     counter++;
   }
 
-  public IntermediateSortTempRow getRow() {
+  public Object[] getRow() {
     return currentRow;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
index cbcbbae..6f05088 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
@@ -19,9 +19,8 @@ package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
 
 public class UnsafeInmemoryHolder implements SortTempChunkHolder {
 
@@ -34,18 +33,21 @@ public class UnsafeInmemoryHolder implements SortTempChunkHolder {
 
   private UnsafeCarbonRowPage rowPage;
 
-  private IntermediateSortTempRow currentRow;
+  private Object[] currentRow;
 
   private long address;
 
-  private IntermediateSortTempRowComparator comparator;
+  private NewRowComparator comparator;
 
-  public UnsafeInmemoryHolder(UnsafeCarbonRowPage rowPage) {
+  private int columnSize;
+
+  public UnsafeInmemoryHolder(UnsafeCarbonRowPage rowPage, int columnSize,
+      int numberOfSortColumns) {
     this.actualSize = rowPage.getBuffer().getActualSize();
     this.rowPage = rowPage;
     LOGGER.audit("Processing unsafe inmemory rows page with size : " + actualSize);
-    this.comparator = new IntermediateSortTempRowComparator(
-        rowPage.getTableFieldStat().getIsSortColNoDictFlags());
+    this.comparator = new NewRowComparator(rowPage.getNoDictionarySortColumnMapping());
+    this.columnSize = columnSize;
   }
 
   public boolean hasNext() {
@@ -56,12 +58,13 @@ public class UnsafeInmemoryHolder implements SortTempChunkHolder {
   }
 
   public void readRow() {
+    currentRow = new Object[columnSize];
     address = rowPage.getBuffer().get(counter);
-    currentRow = rowPage.getRow(address + rowPage.getDataBlock().getBaseOffset());
+    rowPage.getRow(address + rowPage.getDataBlock().getBaseOffset(), currentRow);
     counter++;
   }
 
-  public IntermediateSortTempRow getRow() {
+  public Object[] getRow() {
     return currentRow;
   }
 


[16/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
index 8c8d08f..a689d8e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeFixedLengthDimensionDataChunkStore.java
@@ -124,22 +124,22 @@ public class UnsafeFixedLengthDimensionDataChunkStore
   /**
    * to compare the two byte array
    *
-   * @param index        index of first byte array
+   * @param rowId        index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
     // based on index we need to calculate the actual position in memory block
-    index = index * columnValueSize;
+    rowId = rowId * columnValueSize;
     int compareResult = 0;
     for (int i = 0; i < compareValue.length; i++) {
       compareResult = (CarbonUnsafe.getUnsafe()
-          .getByte(dataPageMemoryBlock.getBaseObject(), dataPageMemoryBlock.getBaseOffset() + index)
+          .getByte(dataPageMemoryBlock.getBaseObject(), dataPageMemoryBlock.getBaseOffset() + rowId)
           & 0xff) - (compareValue[i] & 0xff);
       if (compareResult != 0) {
         break;
       }
-      index++;
+      rowId++;
     }
     return compareResult;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
index 36b2bd8..e1eb378 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/unsafe/UnsafeVariableLengthDimesionDataChunkStore.java
@@ -189,11 +189,11 @@ public class UnsafeVariableLengthDimesionDataChunkStore
   /**
    * to compare the two byte array
    *
-   * @param index index of first byte array
+   * @param rowId index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
     // now to get the row from memory block we need to do following thing
     // 1. first get the current offset
     // 2. if it's not a last row- get the next row offset
@@ -201,13 +201,13 @@ public class UnsafeVariableLengthDimesionDataChunkStore
     // else subtract the current row offset
     // with complete data length get the offset of set of data
     int currentDataOffset = CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(),
-        dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((long)index
+        dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((long) rowId
             * CarbonCommonConstants.INT_SIZE_IN_BYTE * 1L));
     short length = 0;
     // calculating the length of data
-    if (index < numberOfRows - 1) {
+    if (rowId < numberOfRows - 1) {
       int OffsetOfNextdata = CarbonUnsafe.getUnsafe().getInt(dataPageMemoryBlock.getBaseObject(),
-          dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((index + 1)
+          dataPageMemoryBlock.getBaseOffset() + this.dataPointersOffsets + ((rowId + 1)
               * CarbonCommonConstants.INT_SIZE_IN_BYTE));
       length = (short) (OffsetOfNextdata - (currentDataOffset
           + CarbonCommonConstants.SHORT_SIZE_IN_BYTE));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
index 74d268a..e2a4161 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/columnar/ColumnGroupModel.java
@@ -29,13 +29,6 @@ public class ColumnGroupModel {
   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}}
@@ -77,15 +70,6 @@ public class ColumnGroupModel {
   }
 
   /**
-   * 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
@@ -95,16 +79,6 @@ public class ColumnGroupModel {
   }
 
   /**
-   * 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() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
deleted file mode 100644
index 182c8eb..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileHolderImpl.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datastore.impl;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
-
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-public class DFSFileHolderImpl implements FileHolder {
-  /**
-   * cache to hold filename and its stream
-   */
-  private Map<String, FSDataInputStream> fileNameAndStreamCache;
-
-  private String queryId;
-
-  private boolean readPageByPage;
-
-
-  public DFSFileHolderImpl() {
-    this.fileNameAndStreamCache =
-        new HashMap<String, FSDataInputStream>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  @Override public byte[] readByteArray(String filePath, long offset, int length)
-      throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    return read(fileChannel, length, offset);
-  }
-
-  /**
-   * This method will be used to check whether stream is already present in
-   * cache or not for filepath if not present then create it and then add to
-   * cache, other wise get from cache
-   *
-   * @param filePath fully qualified file path
-   * @return channel
-   */
-  public FSDataInputStream updateCache(String filePath) throws IOException {
-    FSDataInputStream fileChannel = fileNameAndStreamCache.get(filePath);
-    if (null == fileChannel) {
-      Path pt = new Path(filePath);
-      FileSystem fs = pt.getFileSystem(FileFactory.getConfiguration());
-      fileChannel = fs.open(pt);
-      fileNameAndStreamCache.put(filePath, fileChannel);
-    }
-    return fileChannel;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and position
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @param offset  position
-   * @return byte buffer
-   */
-  private byte[] read(FSDataInputStream channel, int size, long offset) throws IOException {
-    byte[] byteBffer = new byte[size];
-    channel.seek(offset);
-    channel.readFully(byteBffer);
-    return byteBffer;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and position
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @return byte buffer
-   */
-  private byte[] read(FSDataInputStream channel, int size) throws IOException {
-    byte[] byteBffer = new byte[size];
-    channel.readFully(byteBffer);
-    return byteBffer;
-  }
-
-  @Override public int readInt(String filePath, long offset) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    fileChannel.seek(offset);
-    return fileChannel.readInt();
-  }
-
-  @Override public long readDouble(String filePath, long offset) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    fileChannel.seek(offset);
-    return fileChannel.readLong();
-  }
-
-  @Override public void finish() throws IOException {
-    for (Entry<String, FSDataInputStream> entry : fileNameAndStreamCache.entrySet()) {
-      FSDataInputStream channel = entry.getValue();
-      if (null != channel) {
-        channel.close();
-      }
-    }
-  }
-
-  @Override public byte[] readByteArray(String filePath, int length) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    return read(fileChannel, length);
-  }
-
-  @Override public long readLong(String filePath, long offset) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    fileChannel.seek(offset);
-    return fileChannel.readLong();
-  }
-
-  @Override public int readInt(String filePath) throws IOException {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    return fileChannel.readInt();
-  }
-
-  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
-      throws IOException {
-    byte[] readByteArray = readByteArray(filePath, offset, length);
-    ByteBuffer byteBuffer = ByteBuffer.wrap(readByteArray);
-    byteBuffer.rewind();
-    return byteBuffer;
-  }
-
-  @Override public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  @Override public String getQueryId() {
-    return queryId;
-  }
-
-  @Override public void setReadPageByPage(boolean isReadPageByPage) {
-    this.readPageByPage = isReadPageByPage;
-  }
-
-  @Override public boolean isReadPageByPage() {
-    return readPageByPage;
-  }
-
-  public Map<String, FSDataInputStream> getFileNameAndStreamCache() {
-    return fileNameAndStreamCache;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java
new file mode 100644
index 0000000..1a0cd41
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DFSFileReaderImpl.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datastore.impl;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.FileReader;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class DFSFileReaderImpl implements FileReader {
+  /**
+   * cache to hold filename and its stream
+   */
+  private Map<String, FSDataInputStream> fileNameAndStreamCache;
+
+  private boolean readPageByPage;
+
+  public DFSFileReaderImpl() {
+    this.fileNameAndStreamCache =
+        new HashMap<String, FSDataInputStream>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  @Override public byte[] readByteArray(String filePath, long offset, int length)
+      throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    return read(fileChannel, length, offset);
+  }
+
+  /**
+   * This method will be used to check whether stream is already present in
+   * cache or not for filepath if not present then create it and then add to
+   * cache, other wise get from cache
+   *
+   * @param filePath fully qualified file path
+   * @return channel
+   */
+  private FSDataInputStream updateCache(String filePath) throws IOException {
+    FSDataInputStream fileChannel = fileNameAndStreamCache.get(filePath);
+    if (null == fileChannel) {
+      Path pt = new Path(filePath);
+      FileSystem fs = pt.getFileSystem(FileFactory.getConfiguration());
+      fileChannel = fs.open(pt);
+      fileNameAndStreamCache.put(filePath, fileChannel);
+    }
+    return fileChannel;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and position
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @param offset  position
+   * @return byte buffer
+   */
+  private byte[] read(FSDataInputStream channel, int size, long offset) throws IOException {
+    byte[] byteBffer = new byte[size];
+    channel.seek(offset);
+    channel.readFully(byteBffer);
+    return byteBffer;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and position
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @return byte buffer
+   */
+  private byte[] read(FSDataInputStream channel, int size) throws IOException {
+    byte[] byteBffer = new byte[size];
+    channel.readFully(byteBffer);
+    return byteBffer;
+  }
+
+  @Override public int readInt(String filePath, long offset) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    fileChannel.seek(offset);
+    return fileChannel.readInt();
+  }
+
+  @Override public long readDouble(String filePath, long offset) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    fileChannel.seek(offset);
+    return fileChannel.readLong();
+  }
+
+  @Override public void finish() throws IOException {
+    for (Entry<String, FSDataInputStream> entry : fileNameAndStreamCache.entrySet()) {
+      FSDataInputStream channel = entry.getValue();
+      if (null != channel) {
+        channel.close();
+      }
+    }
+  }
+
+  @Override public byte[] readByteArray(String filePath, int length) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    return read(fileChannel, length);
+  }
+
+  @Override public long readLong(String filePath, long offset) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    fileChannel.seek(offset);
+    return fileChannel.readLong();
+  }
+
+  @Override public int readInt(String filePath) throws IOException {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    return fileChannel.readInt();
+  }
+
+  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
+      throws IOException {
+    byte[] readByteArray = readByteArray(filePath, offset, length);
+    ByteBuffer byteBuffer = ByteBuffer.wrap(readByteArray);
+    byteBuffer.rewind();
+    return byteBuffer;
+  }
+
+  @Override public void setReadPageByPage(boolean isReadPageByPage) {
+    this.readPageByPage = isReadPageByPage;
+  }
+
+  @Override public boolean isReadPageByPage() {
+    return readPageByPage;
+  }
+
+  public Map<String, FSDataInputStream> getFileNameAndStreamCache() {
+    return fileNameAndStreamCache;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
index 67648fe..b58a473 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
@@ -17,24 +17,28 @@
 
 package org.apache.carbondata.core.datastore.impl;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.filesystem.*;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.filesystem.AlluxioCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.HDFSCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.ViewFSCarbonFile;
 
 import org.apache.hadoop.conf.Configuration;
 
 public class DefaultFileTypeProvider implements FileTypeInerface {
 
-  public FileHolder getFileHolder(FileFactory.FileType fileType) {
+  public FileReader getFileHolder(FileFactory.FileType fileType) {
     switch (fileType) {
       case LOCAL:
-        return new FileHolderImpl();
+        return new FileReaderImpl();
       case HDFS:
       case ALLUXIO:
       case VIEWFS:
       case S3:
-        return new DFSFileHolderImpl();
+        return new DFSFileReaderImpl();
       default:
-        return new FileHolderImpl();
+        return new FileReaderImpl();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
index f141991..9bcdfae 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
@@ -28,7 +28,7 @@ import java.nio.channels.FileChannel;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 
 import org.apache.commons.io.FileUtils;
@@ -62,7 +62,7 @@ public final class FileFactory {
     return configuration;
   }
 
-  public static FileHolder getFileHolder(FileType fileType) {
+  public static FileReader getFileHolder(FileType fileType) {
     return fileFileTypeInerface.getFileHolder(fileType);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
deleted file mode 100644
index cc589b7..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileHolderImpl.java
+++ /dev/null
@@ -1,224 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.datastore.impl;
-
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
-
-public class FileHolderImpl implements FileHolder {
-  /**
-   * cache to hold filename and its stream
-   */
-  private Map<String, FileChannel> fileNameAndStreamCache;
-  private String queryId;
-
-  private boolean readPageByPage;
-
-  /**
-   * FileHolderImpl Constructor
-   * It will create the cache
-   */
-  public FileHolderImpl() {
-    this.fileNameAndStreamCache =
-        new HashMap<String, FileChannel>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  public FileHolderImpl(int capacity) {
-    this.fileNameAndStreamCache = new HashMap<String, FileChannel>(capacity);
-  }
-
-  /**
-   * This method will be used to read the byte array from file based on offset
-   * and length(number of bytes) need to read
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @param length   number of bytes to be read
-   * @return read byte array
-   */
-  @Override public byte[] readByteArray(String filePath, long offset, int length)
-      throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, length, offset);
-    return byteBffer.array();
-  }
-
-  /**
-   * This method will be used to close all the streams currently present in the cache
-   */
-  @Override public void finish() throws IOException {
-    for (Entry<String, FileChannel> entry : fileNameAndStreamCache.entrySet()) {
-      FileChannel channel = entry.getValue();
-      if (null != channel) {
-        channel.close();
-      }
-    }
-  }
-
-  /**
-   * This method will be used to read int from file from postion(offset), here
-   * length will be always 4 bacause int byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read int
-   */
-  @Override public int readInt(String filePath, long offset) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.INT_SIZE_IN_BYTE, offset);
-    return byteBffer.getInt();
-  }
-
-  /**
-   * This method will be used to read int from file from postion(offset), here
-   * length will be always 4 bacause int byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @return read int
-   */
-  @Override public int readInt(String filePath) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.INT_SIZE_IN_BYTE);
-    return byteBffer.getInt();
-  }
-
-  /**
-   * This method will be used to read int from file from postion(offset), here
-   * length will be always 4 bacause int byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read int
-   */
-  @Override public long readDouble(String filePath, long offset) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
-    return byteBffer.getLong();
-  }
-
-  /**
-   * This method will be used to check whether stream is already present in
-   * cache or not for filepath if not present then create it and then add to
-   * cache, other wise get from cache
-   *
-   * @param filePath fully qualified file path
-   * @return channel
-   */
-  private FileChannel updateCache(String filePath) throws FileNotFoundException {
-    FileChannel fileChannel = fileNameAndStreamCache.get(filePath);
-    if (null == fileChannel) {
-      FileInputStream stream = new FileInputStream(filePath);
-      fileChannel = stream.getChannel();
-      fileNameAndStreamCache.put(filePath, fileChannel);
-    }
-    return fileChannel;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and position
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @param offset  position
-   * @return byte buffer
-   */
-  private ByteBuffer read(FileChannel channel, int size, long offset) throws IOException {
-    ByteBuffer byteBffer = ByteBuffer.allocate(size);
-    channel.position(offset);
-    channel.read(byteBffer);
-    byteBffer.rewind();
-    return byteBffer;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and position
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @return byte buffer
-   */
-  private ByteBuffer read(FileChannel channel, int size) throws IOException {
-    ByteBuffer byteBffer = ByteBuffer.allocate(size);
-    channel.read(byteBffer);
-    byteBffer.rewind();
-    return byteBffer;
-  }
-
-
-  /**
-   * This method will be used to read the byte array from file based on length(number of bytes)
-   *
-   * @param filePath fully qualified file path
-   * @param length   number of bytes to be read
-   * @return read byte array
-   */
-  @Override public byte[] readByteArray(String filePath, int length) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, length);
-    return byteBffer.array();
-  }
-
-  /**
-   * This method will be used to read long from file from postion(offset), here
-   * length will be always 8 bacause int byte size is 8
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read long
-   */
-  @Override public long readLong(String filePath, long offset) throws IOException {
-    FileChannel fileChannel = updateCache(filePath);
-    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
-    return byteBffer.getLong();
-  }
-
-  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
-      throws IOException {
-    ByteBuffer byteBuffer = ByteBuffer.allocate(length);
-    FileChannel fileChannel = updateCache(filePath);
-    fileChannel.position(offset);
-    fileChannel.read(byteBuffer);
-    byteBuffer.rewind();
-    return byteBuffer;
-  }
-
-  @Override public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  @Override public String getQueryId() {
-    return queryId;
-  }
-
-  @Override public void setReadPageByPage(boolean isReadPageByPage) {
-    this.readPageByPage = isReadPageByPage;
-  }
-
-  @Override public boolean isReadPageByPage() {
-    return readPageByPage;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java
new file mode 100644
index 0000000..6fef278
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileReaderImpl.java
@@ -0,0 +1,215 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.datastore.impl;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.FileReader;
+
+public class FileReaderImpl implements FileReader {
+  /**
+   * cache to hold filename and its stream
+   */
+  private Map<String, FileChannel> fileNameAndStreamCache;
+
+  private boolean readPageByPage;
+
+  /**
+   * FileReaderImpl Constructor
+   * It will create the cache
+   */
+  public FileReaderImpl() {
+    this.fileNameAndStreamCache =
+        new HashMap<String, FileChannel>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  public FileReaderImpl(int capacity) {
+    this.fileNameAndStreamCache = new HashMap<String, FileChannel>(capacity);
+  }
+
+  /**
+   * This method will be used to read the byte array from file based on offset
+   * and length(number of bytes) need to read
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @param length   number of bytes to be read
+   * @return read byte array
+   */
+  @Override public byte[] readByteArray(String filePath, long offset, int length)
+      throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, length, offset);
+    return byteBffer.array();
+  }
+
+  /**
+   * This method will be used to close all the streams currently present in the cache
+   */
+  @Override public void finish() throws IOException {
+    for (Entry<String, FileChannel> entry : fileNameAndStreamCache.entrySet()) {
+      FileChannel channel = entry.getValue();
+      if (null != channel) {
+        channel.close();
+      }
+    }
+  }
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read int
+   */
+  @Override public int readInt(String filePath, long offset) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.INT_SIZE_IN_BYTE, offset);
+    return byteBffer.getInt();
+  }
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @return read int
+   */
+  @Override public int readInt(String filePath) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.INT_SIZE_IN_BYTE);
+    return byteBffer.getInt();
+  }
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read int
+   */
+  @Override public long readDouble(String filePath, long offset) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
+    return byteBffer.getLong();
+  }
+
+  /**
+   * This method will be used to check whether stream is already present in
+   * cache or not for filepath if not present then create it and then add to
+   * cache, other wise get from cache
+   *
+   * @param filePath fully qualified file path
+   * @return channel
+   */
+  private FileChannel updateCache(String filePath) throws FileNotFoundException {
+    FileChannel fileChannel = fileNameAndStreamCache.get(filePath);
+    if (null == fileChannel) {
+      FileInputStream stream = new FileInputStream(filePath);
+      fileChannel = stream.getChannel();
+      fileNameAndStreamCache.put(filePath, fileChannel);
+    }
+    return fileChannel;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and position
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @param offset  position
+   * @return byte buffer
+   */
+  private ByteBuffer read(FileChannel channel, int size, long offset) throws IOException {
+    ByteBuffer byteBffer = ByteBuffer.allocate(size);
+    channel.position(offset);
+    channel.read(byteBffer);
+    byteBffer.rewind();
+    return byteBffer;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and position
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @return byte buffer
+   */
+  private ByteBuffer read(FileChannel channel, int size) throws IOException {
+    ByteBuffer byteBffer = ByteBuffer.allocate(size);
+    channel.read(byteBffer);
+    byteBffer.rewind();
+    return byteBffer;
+  }
+
+
+  /**
+   * This method will be used to read the byte array from file based on length(number of bytes)
+   *
+   * @param filePath fully qualified file path
+   * @param length   number of bytes to be read
+   * @return read byte array
+   */
+  @Override public byte[] readByteArray(String filePath, int length) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, length);
+    return byteBffer.array();
+  }
+
+  /**
+   * This method will be used to read long from file from postion(offset), here
+   * length will be always 8 bacause int byte size is 8
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read long
+   */
+  @Override public long readLong(String filePath, long offset) throws IOException {
+    FileChannel fileChannel = updateCache(filePath);
+    ByteBuffer byteBffer = read(fileChannel, CarbonCommonConstants.LONG_SIZE_IN_BYTE, offset);
+    return byteBffer.getLong();
+  }
+
+  @Override public ByteBuffer readByteBuffer(String filePath, long offset, int length)
+      throws IOException {
+    ByteBuffer byteBuffer = ByteBuffer.allocate(length);
+    FileChannel fileChannel = updateCache(filePath);
+    fileChannel.position(offset);
+    fileChannel.read(byteBuffer);
+    byteBuffer.rewind();
+    return byteBuffer;
+  }
+
+  @Override public void setReadPageByPage(boolean isReadPageByPage) {
+    this.readPageByPage = isReadPageByPage;
+  }
+
+  @Override public boolean isReadPageByPage() {
+    return readPageByPage;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java
index 4676278..413261c 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileTypeInerface.java
@@ -17,14 +17,14 @@
 
 package org.apache.carbondata.core.datastore.impl;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 
 import org.apache.hadoop.conf.Configuration;
 
 public interface FileTypeInerface {
 
-  FileHolder getFileHolder(FileFactory.FileType fileType);
+  FileReader getFileHolder(FileFactory.FileType fileType);
   CarbonFile getCarbonFile(String path, FileFactory.FileType fileType);
   CarbonFile getCarbonFile(String path, FileFactory.FileType fileType, Configuration configuration);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
index 19b1f1c..fe4cf83 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
@@ -18,9 +18,8 @@ package org.apache.carbondata.core.datastore.impl.btree;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
 import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
@@ -31,38 +30,31 @@ import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 public abstract class AbstractBTreeLeafNode implements BTreeNode {
 
   /**
-   * Below method will be used to load the data block
-   *
-   * @param blockInfo block detail
-   */
-  protected BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache;
-
-  /**
    * number of keys in a btree
    */
-  protected int numberOfKeys;
+  int numberOfKeys;
 
   /**
    * node number
    */
-  protected long nodeNumber;
+  long nodeNumber;
 
   /**
    * Next node of the leaf
    */
-  protected BTreeNode nextNode;
+  private BTreeNode nextNode;
 
   /**
    * max key of the column this will be used to check whether this leaf will
    * be used for scanning or not
    */
-  protected byte[][] maxKeyOfColumns;
+  byte[][] maxKeyOfColumns;
 
   /**
    * min key of the column this will be used to check whether this leaf will
    * be used for scanning or not
    */
-  protected byte[][] minKeyOfColumns;
+  byte[][] minKeyOfColumns;
 
   /**
    * Method to get the next block this can be used while scanning when
@@ -70,7 +62,7 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    *
    * @return next block
    */
-  @Override public int nodeSize() {
+  @Override public int numRows() {
     return this.numberOfKeys;
   }
 
@@ -109,7 +101,7 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    *
    * @return block number
    */
-  @Override public long nodeNumber() {
+  @Override public long nodeIndex() {
     return nodeNumber;
   }
 
@@ -174,11 +166,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * Below method will be used to get the dimension chunks
    *
    * @param fileReader   file reader to read the chunks from file
-   * @param blockIndexes indexes of the blocks need to be read
+   * @param columnIndexRange indexes of the blocks need to be read
    * @return dimension data chunks
    */
-  @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     // No required here as leaf which will will be use this class will implement its own get
     // dimension chunks
     return null;
@@ -188,11 +180,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * Below method will be used to get the dimension chunk
    *
    * @param fileReader file reader to read the chunk from file
-   * @param blockIndex block index to be read
+   * @param columnIndex block index to be read
    * @return dimension data chunk
    */
-  @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader,
-      int blockIndex) throws IOException {
+  @Override public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader,
+      int columnIndex) throws IOException {
     // No required here as leaf which will will be use this class will implement
     // its own get dimension chunks
     return null;
@@ -202,11 +194,11 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes block indexes to be read from file
+   * @param columnIndexRange block indexes to be read from file
    * @return measure column data chunk
    */
-  @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     // No required here as leaf which will will be use this class will implement its own get
     // measure chunks
     return null;
@@ -216,30 +208,16 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
    * Below method will be used to read the measure chunk
    *
    * @param fileReader file read to read the file chunk
-   * @param blockIndex block index to be read from file
+   * @param columnIndex block index to be read from file
    * @return measure data chunk
    */
-  @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex)
+  @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     // No required here as leaf which will will be use this class will implement its own get
     // measure chunks
     return null;
   }
 
-  /**
-   * @param deleteDeltaDataCache
-   */
-  public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) {
-
-    this.deleteDeltaDataCache = deleteDeltaDataCache;
-  }
-  /**
-   * @return the segmentProperties
-   */
-  public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() {
-    return deleteDeltaDataCache;
-  }
-
   @Override
   public int getPageRowCount(int pageNumber) {
     throw new UnsupportedOperationException("Unsupported operation");

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
index 2f8aadf..688d56a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeDataRefNodeFinder.java
@@ -106,7 +106,7 @@ public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
   private BTreeNode findFirstLeafNode(IndexKey key, BTreeNode node) {
     int childNodeIndex;
     int low = 0;
-    int high = node.nodeSize() - 1;
+    int high = node.numRows() - 1;
     int mid = 0;
     int compareRes = -1;
     IndexKey[] nodeKeys = node.getNodeKeys();
@@ -156,7 +156,7 @@ public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
   private BTreeNode findLastLeafNode(IndexKey key, BTreeNode node) {
     int childNodeIndex;
     int low = 0;
-    int high = node.nodeSize() - 1;
+    int high = node.numRows() - 1;
     int mid = 0;
     int compareRes = -1;
     IndexKey[] nodeKeys = node.getNodeKeys();
@@ -172,7 +172,7 @@ public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
       } else {
         int currentPos = mid;
         // if key is matched then get the first entry
-        while (currentPos + 1 < node.nodeSize()
+        while (currentPos + 1 < node.numRows()
             && compareIndexes(key, nodeKeys[currentPos + 1]) == 0) {
           currentPos++;
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
index ccc5e12..c200f8d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
@@ -19,10 +19,9 @@ package org.apache.carbondata.core.datastore.impl.btree;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
@@ -34,13 +33,6 @@ import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 public class BTreeNonLeafNode implements BTreeNode {
 
   /**
-   * Below method will be used to load the data block
-   *
-   * @param blockInfo block detail
-   */
-  protected BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache;
-
-  /**
    * Child nodes
    */
   private BTreeNode[] children;
@@ -50,7 +42,7 @@ public class BTreeNonLeafNode implements BTreeNode {
    */
   private List<IndexKey> listOfKeys;
 
-  public BTreeNonLeafNode() {
+  BTreeNonLeafNode() {
     // creating a list which will store all the indexes
     listOfKeys = new ArrayList<IndexKey>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
   }
@@ -120,7 +112,7 @@ public class BTreeNonLeafNode implements BTreeNode {
    *
    * @return number of keys in the block
    */
-  @Override public int nodeSize() {
+  @Override public int numRows() {
     return listOfKeys.size();
   }
 
@@ -131,11 +123,11 @@ public class BTreeNonLeafNode implements BTreeNode {
    *
    * @return block number
    */
-  @Override public long nodeNumber() {
+  @Override public long nodeIndex() {
     throw new UnsupportedOperationException("Unsupported operation");
   }
 
-  @Override public String blockletId() {
+  @Override public short blockletIndex() {
     throw new UnsupportedOperationException("Unsupported operation");
   }
 
@@ -171,11 +163,11 @@ public class BTreeNonLeafNode implements BTreeNode {
    * Below method will be used to get the dimension chunks
    *
    * @param fileReader   file reader to read the chunks from file
-   * @param blockIndexes indexes of the blocks need to be read
+   * @param columnIndexRange indexes of the blocks need to be read
    * @return dimension data chunks
    */
-  @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader,
-      int[][] blockIndexes) {
+  @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader,
+      int[][] columnIndexRange) {
 
     // operation of getting the dimension chunks is not supported as its a
     // non leaf node
@@ -191,8 +183,8 @@ public class BTreeNonLeafNode implements BTreeNode {
    * @param fileReader file reader to read the chunk from file
    * @return dimension data chunk
    */
-  @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader,
-      int blockIndexes) {
+  @Override public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader,
+      int columnIndex) {
     // operation of getting the dimension chunk is not supported as its a
     // non leaf node
     // and in case of B+Tree data will be stored only in leaf node and
@@ -205,11 +197,11 @@ public class BTreeNonLeafNode implements BTreeNode {
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes block indexes to be read from file
+   * @param columnIndexRange block indexes to be read from file
    * @return measure column data chunk
    */
-  @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader,
-      int[][] blockIndexes) {
+  @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) {
     // operation of getting the measure chunk is not supported as its a non
     // leaf node
     // and in case of B+Tree data will be stored only in leaf node and
@@ -222,11 +214,11 @@ public class BTreeNonLeafNode implements BTreeNode {
    * Below method will be used to read the measure chunk
    *
    * @param fileReader file read to read the file chunk
-   * @param blockIndex block index to be read from file
+   * @param columnIndex block index to be read from file
    * @return measure data chunk
    */
 
-  @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) {
+  @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) {
     // operation of getting the measure chunk is not supported as its a non
     // leaf node
     // and in case of B+Tree data will be stored only in leaf node and
@@ -236,20 +228,6 @@ public class BTreeNonLeafNode implements BTreeNode {
   }
 
   /**
-   * @return the segmentProperties
-   */
-  public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) {
-
-    this.deleteDeltaDataCache = deleteDeltaDataCache;
-  }
-  /**
-   * @return the segmentProperties
-   */
-  public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() {
-    return deleteDeltaDataCache;
-  }
-
-  /**
    * number of pages in blocklet
    * @return
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
index 25817f5..8af7eae 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockBTreeLeafNode.java
@@ -38,7 +38,7 @@ public class BlockBTreeLeafNode extends AbstractBTreeLeafNode {
    *                      node
    * @param metadataIndex metadata index
    */
-  public BlockBTreeLeafNode(BTreeBuilderInfo builderInfos, int metadataIndex, long nodeNumber) {
+  BlockBTreeLeafNode(BTreeBuilderInfo builderInfos, int metadataIndex, long nodeNumber) {
     DataFileFooter footer = builderInfos.getFooterList().get(metadataIndex);
     BlockletMinMaxIndex minMaxIndex = footer.getBlockletIndex().getMinMaxIndex();
     maxKeyOfColumns = minMaxIndex.getMaxValues();
@@ -63,8 +63,8 @@ public class BlockBTreeLeafNode extends AbstractBTreeLeafNode {
    * Below method is suppose to return the Blocklet ID.
    * @return
    */
-  @Override public String blockletId() {
-    return blockInfo.getTableBlockInfo().getDetailInfo().getBlockletId().toString();
+  @Override public short blockletIndex() {
+    return blockInfo.getTableBlockInfo().getDetailInfo().getBlockletId();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
index 94221ba..ddd7fcf 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BlockletBTreeLeafNode.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.datastore.BTreeBuilderInfo;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
@@ -69,7 +69,7 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    *                     this will be used during query execution when we can
    *                     give some leaf node of a btree to one executor some to other
    */
-  public BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long nodeNumber) {
+  BlockletBTreeLeafNode(BTreeBuilderInfo builderInfos, int leafIndex, long nodeNumber) {
     // get a lead node min max
     BlockletMinMaxIndex minMaxIndex =
         builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex).getBlockletIndex()
@@ -124,23 +124,23 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
     }
   }
 
-  @Override public String blockletId() {
-    return "0";
+  @Override public short blockletIndex() {
+    return 0;
   }
 
   /**
    * Below method will be used to get the dimension chunks
    *
    * @param fileReader   file reader to read the chunks from file
-   * @param blockIndexes indexes of the blocks need to be read
+   * @param columnIndexRange indexes of the blocks need to be read
    * @return dimension data chunks
    */
-  @Override public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     if (fileReader.isReadPageByPage()) {
-      return dimensionChunksPageLevelReader.readRawDimensionChunks(fileReader, blockIndexes);
+      return dimensionChunksPageLevelReader.readRawDimensionChunks(fileReader, columnIndexRange);
     } else {
-      return dimensionChunksReader.readRawDimensionChunks(fileReader, blockIndexes);
+      return dimensionChunksReader.readRawDimensionChunks(fileReader, columnIndexRange);
     }
   }
 
@@ -148,15 +148,15 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * Below method will be used to get the dimension chunk
    *
    * @param fileReader file reader to read the chunk from file
-   * @param blockIndex block index to be read
+   * @param columnIndex block index to be read
    * @return dimension data chunk
    */
-  @Override public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndex)
-      throws IOException {
+  @Override public DimensionRawColumnChunk readDimensionChunk(
+      FileReader fileReader, int columnIndex) throws IOException {
     if (fileReader.isReadPageByPage()) {
-      return dimensionChunksPageLevelReader.readRawDimensionChunk(fileReader, blockIndex);
+      return dimensionChunksPageLevelReader.readRawDimensionChunk(fileReader, columnIndex);
     } else {
-      return dimensionChunksReader.readRawDimensionChunk(fileReader, blockIndex);
+      return dimensionChunksReader.readRawDimensionChunk(fileReader, columnIndex);
     }
   }
 
@@ -164,15 +164,15 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes block indexes to be read from file
+   * @param columnIndexRange block indexes to be read from file
    * @return measure column data chunk
    */
-  @Override public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     if (fileReader.isReadPageByPage()) {
-      return measureColumnChunkPageLevelReader.readRawMeasureChunks(fileReader, blockIndexes);
+      return measureColumnChunkPageLevelReader.readRawMeasureChunks(fileReader, columnIndexRange);
     } else {
-      return measureColumnChunkReader.readRawMeasureChunks(fileReader, blockIndexes);
+      return measureColumnChunkReader.readRawMeasureChunks(fileReader, columnIndexRange);
     }
   }
 
@@ -180,15 +180,15 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
    * Below method will be used to read the measure chunk
    *
    * @param fileReader file read to read the file chunk
-   * @param blockIndex block index to be read from file
+   * @param columnIndex block index to be read from file
    * @return measure data chunk
    */
-  @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex)
+  @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     if (fileReader.isReadPageByPage()) {
-      return measureColumnChunkPageLevelReader.readRawMeasureChunk(fileReader, blockIndex);
+      return measureColumnChunkPageLevelReader.readRawMeasureChunk(fileReader, columnIndex);
     } else {
-      return measureColumnChunkReader.readRawMeasureChunk(fileReader, blockIndex);
+      return measureColumnChunkReader.readRawMeasureChunk(fileReader, columnIndex);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
index 6d96b3b..597def0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/EncodingFactory.java
@@ -41,7 +41,13 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.format.Encoding;
 
-import static org.apache.carbondata.format.Encoding.*;
+import static org.apache.carbondata.format.Encoding.ADAPTIVE_DELTA_FLOATING;
+import static org.apache.carbondata.format.Encoding.ADAPTIVE_DELTA_INTEGRAL;
+import static org.apache.carbondata.format.Encoding.ADAPTIVE_FLOATING;
+import static org.apache.carbondata.format.Encoding.ADAPTIVE_INTEGRAL;
+import static org.apache.carbondata.format.Encoding.BOOL_BYTE;
+import static org.apache.carbondata.format.Encoding.DIRECT_COMPRESS;
+import static org.apache.carbondata.format.Encoding.RLE_INTEGRAL;
 
 /**
  * Base class for encoding factory implementation.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
index c7411d6..daba470 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/server/NonSecureDictionaryServer.java
@@ -27,7 +27,6 @@ import org.apache.carbondata.core.dictionary.service.AbstractDictionaryServer;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.util.CarbonProperties;
 
-
 import io.netty.bootstrap.ServerBootstrap;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelOption;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
index ce05fe2..2865d4b 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailInfo.java
@@ -213,10 +213,6 @@ public class BlockletDetailInfo implements Serializable, Writable {
     return columnSchemas;
   }
 
-  public void setColumnSchemas(List<ColumnSchema> columnSchemas) {
-    this.columnSchemas = columnSchemas;
-  }
-
   public void setColumnSchemaBinary(byte[] columnSchemaBinary) {
     this.columnSchemaBinary = columnSchemaBinary;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
new file mode 100644
index 0000000..5cd59cb
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
@@ -0,0 +1,228 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.indexstore.blockletindex;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
+import org.apache.carbondata.core.constants.CarbonVersionConstants;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
+import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
+import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
+import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
+
+/**
+ * wrapper for blocklet data map data
+ */
+public class BlockletDataRefNode implements DataRefNode {
+
+  private List<TableBlockInfo> blockInfos;
+
+  private int index;
+
+  private int[] dimensionLens;
+
+  BlockletDataRefNode(List<TableBlockInfo> blockInfos, int index, int[] dimensionLens) {
+    this.blockInfos = blockInfos;
+    // Update row count and page count to blocklet info
+    for (TableBlockInfo blockInfo : blockInfos) {
+      BlockletDetailInfo detailInfo = blockInfo.getDetailInfo();
+      detailInfo.getBlockletInfo().setNumberOfRows(detailInfo.getRowCount());
+      detailInfo.getBlockletInfo().setNumberOfPages(detailInfo.getPagesCount());
+      detailInfo.setBlockletId(blockInfo.getDetailInfo().getBlockletId());
+      int[] pageRowCount = new int[detailInfo.getPagesCount()];
+      int numberOfPagesCompletelyFilled = detailInfo.getRowCount();
+      // no. of rows to a page is 120000 in V2 and 32000 in V3, same is handled to get the number
+      // of pages filled
+      if (blockInfo.getVersion() == ColumnarFormatVersion.V2) {
+        numberOfPagesCompletelyFilled /=
+            CarbonVersionConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT_V2;
+      } else {
+        numberOfPagesCompletelyFilled /=
+            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      }
+      int lastPageRowCount = detailInfo.getRowCount()
+          % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      for (int i = 0; i < numberOfPagesCompletelyFilled; i++) {
+        pageRowCount[i] =
+            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      }
+      if (lastPageRowCount > 0) {
+        pageRowCount[pageRowCount.length - 1] = lastPageRowCount;
+      }
+      detailInfo.getBlockletInfo().setNumberOfRowsPerPage(pageRowCount);
+    }
+    this.index = index;
+    this.dimensionLens = dimensionLens;
+  }
+
+  @Override public DataRefNode getNextDataRefNode() {
+    if (index + 1 < blockInfos.size()) {
+      return new BlockletDataRefNode(blockInfos, index + 1, dimensionLens);
+    }
+    return null;
+  }
+
+  @Override public int numRows() {
+    return blockInfos.get(index).getDetailInfo().getRowCount();
+  }
+
+  @Override public long nodeIndex() {
+    return index;
+  }
+
+  @Override public short blockletIndex() {
+    return blockInfos.get(index).getDetailInfo().getBlockletId();
+  }
+
+  @Override
+  public byte[][] getColumnsMaxValue() {
+    BlockletIndex blockletIndex =
+        blockInfos.get(index).getDetailInfo().getBlockletInfo().getBlockletIndex();
+    // In case of blocklet distribution this will be null
+    if (null != blockletIndex) {
+      return blockletIndex.getMinMaxIndex().getMaxValues();
+    }
+    return null;
+  }
+
+  @Override
+  public byte[][] getColumnsMinValue() {
+    BlockletIndex blockletIndex =
+        blockInfos.get(index).getDetailInfo().getBlockletInfo().getBlockletIndex();
+    // In case of blocklet distribution this will be null
+    if (null != blockletIndex) {
+      return blockletIndex.getMinMaxIndex().getMinValues();
+    }
+    return null;
+  }
+
+  @Override
+  public DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, int[][] blockIndexes)
+      throws IOException {
+    DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader);
+    return dimensionChunksReader.readRawDimensionChunks(fileReader, blockIndexes);
+  }
+
+  @Override
+  public DimensionRawColumnChunk readDimensionChunk(FileReader fileReader, int columnIndex)
+      throws IOException {
+    DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader);
+    return dimensionChunksReader.readRawDimensionChunk(fileReader, columnIndex);
+  }
+
+  @Override
+  public MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, int[][] columnIndexRange)
+      throws IOException {
+    MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
+    MeasureRawColumnChunk[] measureRawColumnChunks =
+        measureColumnChunkReader.readRawMeasureChunks(fileReader, columnIndexRange);
+    updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunks);
+    return measureRawColumnChunks;
+  }
+
+  @Override public MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex)
+      throws IOException {
+    MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
+    MeasureRawColumnChunk measureRawColumnChunk =
+        measureColumnChunkReader.readRawMeasureChunk(fileReader, columnIndex);
+    updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunk);
+    return measureRawColumnChunk;
+  }
+
+  /**
+   * This method is written specifically for old store wherein the measure min and max values
+   * are written opposite (i.e min in place of max and amx in place of min). Due to this computing
+   * f measure filter with current code is impacted. In order to sync with current min and
+   * max values only in case old store and measures is reversed
+   *
+   * @param measureRawColumnChunk
+   */
+  private void updateMeasureRawColumnChunkMinMaxValues(
+      MeasureRawColumnChunk measureRawColumnChunk) {
+    if (blockInfos.get(index).isDataBlockFromOldStore()) {
+      byte[][] maxValues = measureRawColumnChunk.getMaxValues();
+      byte[][] minValues = measureRawColumnChunk.getMinValues();
+      measureRawColumnChunk.setMaxValues(minValues);
+      measureRawColumnChunk.setMinValues(maxValues);
+    }
+  }
+
+  private void updateMeasureRawColumnChunkMinMaxValues(
+      MeasureRawColumnChunk[] measureRawColumnChunks) {
+    if (blockInfos.get(index).isDataBlockFromOldStore()) {
+      for (int i = 0; i < measureRawColumnChunks.length; i++) {
+        if (null != measureRawColumnChunks[i]) {
+          updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunks[i]);
+        }
+      }
+    }
+  }
+
+  private DimensionColumnChunkReader getDimensionColumnChunkReader(FileReader fileReader) {
+    ColumnarFormatVersion version =
+        ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
+    if (fileReader.isReadPageByPage()) {
+      return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
+          blockInfos.get(index).getFilePath(), true);
+    } else {
+      return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
+          blockInfos.get(index).getFilePath(), false);
+    }
+  }
+
+  private MeasureColumnChunkReader getMeasureColumnChunkReader(FileReader fileReader) {
+    ColumnarFormatVersion version =
+        ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
+    if (fileReader.isReadPageByPage()) {
+      return CarbonDataReaderFactory.getInstance().getMeasureColumnChunkReader(version,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
+          blockInfos.get(index).getFilePath(), true);
+    } else {
+      return CarbonDataReaderFactory.getInstance().getMeasureColumnChunkReader(version,
+          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
+          blockInfos.get(index).getFilePath(), false);
+    }
+  }
+
+  @Override public int numberOfPages() {
+    return blockInfos.get(index).getDetailInfo().getPagesCount();
+  }
+
+  @Override public int getPageRowCount(int pageNumber) {
+    return blockInfos.get(index).getDetailInfo().getBlockletInfo()
+        .getNumberOfRowsPerPage()[pageNumber];
+  }
+
+  public int numberOfNodes() {
+    return blockInfos.size();
+  }
+
+  public List<TableBlockInfo> getBlockInfos() {
+    return blockInfos;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java
deleted file mode 100644
index 4e49ede..0000000
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNodeWrapper.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.indexstore.blockletindex;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
-import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
-import org.apache.carbondata.core.constants.CarbonVersionConstants;
-import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory;
-import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
-import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
-import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
-
-/**
- * wrapper for blocklet data map data
- */
-public class BlockletDataRefNodeWrapper implements DataRefNode {
-
-  private List<TableBlockInfo> blockInfos;
-
-  private int index;
-
-  private int[] dimensionLens;
-
-  private BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache;
-
-  public BlockletDataRefNodeWrapper(List<TableBlockInfo> blockInfos, int index,
-      int[] dimensionLens) {
-    this.blockInfos = blockInfos;
-    // Update row count and page count to blocklet info
-    for (TableBlockInfo blockInfo : blockInfos) {
-      BlockletDetailInfo detailInfo = blockInfo.getDetailInfo();
-      detailInfo.getBlockletInfo().setNumberOfRows(detailInfo.getRowCount());
-      detailInfo.getBlockletInfo().setNumberOfPages(detailInfo.getPagesCount());
-      detailInfo.setBlockletId(blockInfo.getDetailInfo().getBlockletId());
-      int[] pageRowCount = new int[detailInfo.getPagesCount()];
-      int numberOfPagesCompletelyFilled = detailInfo.getRowCount();
-      // no. of rows to a page is 120000 in V2 and 32000 in V3, same is handled to get the number
-      // of pages filled
-      if (blockInfo.getVersion() == ColumnarFormatVersion.V2) {
-        numberOfPagesCompletelyFilled /=
-            CarbonVersionConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT_V2;
-      } else {
-        numberOfPagesCompletelyFilled /=
-            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      }
-      int lastPageRowCount = detailInfo.getRowCount()
-          % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      for (int i = 0; i < numberOfPagesCompletelyFilled; i++) {
-        pageRowCount[i] =
-            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      }
-      if (lastPageRowCount > 0) {
-        pageRowCount[pageRowCount.length - 1] = lastPageRowCount;
-      }
-      detailInfo.getBlockletInfo().setNumberOfRowsPerPage(pageRowCount);
-    }
-    this.index = index;
-    this.dimensionLens = dimensionLens;
-  }
-
-  @Override public DataRefNode getNextDataRefNode() {
-    if (index + 1 < blockInfos.size()) {
-      return new BlockletDataRefNodeWrapper(blockInfos, index + 1, dimensionLens);
-    }
-    return null;
-  }
-
-  @Override public int nodeSize() {
-    return blockInfos.get(index).getDetailInfo().getRowCount();
-  }
-
-  @Override public long nodeNumber() {
-    return index;
-  }
-
-  @Override public String blockletId() {
-    return blockInfos.get(index).getDetailInfo().getBlockletId().toString();
-  }
-
-  @Override
-  public byte[][] getColumnsMaxValue() {
-    BlockletIndex blockletIndex =
-        blockInfos.get(index).getDetailInfo().getBlockletInfo().getBlockletIndex();
-    // In case of blocklet distribution this will be null
-    if (null != blockletIndex) {
-      return blockletIndex.getMinMaxIndex().getMaxValues();
-    }
-    return null;
-  }
-
-  @Override
-  public byte[][] getColumnsMinValue() {
-    BlockletIndex blockletIndex =
-        blockInfos.get(index).getDetailInfo().getBlockletInfo().getBlockletIndex();
-    // In case of blocklet distribution this will be null
-    if (null != blockletIndex) {
-      return blockletIndex.getMinMaxIndex().getMinValues();
-    }
-    return null;
-  }
-
-  @Override
-  public DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, int[][] blockIndexes)
-      throws IOException {
-    DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader);
-    return dimensionChunksReader.readRawDimensionChunks(fileReader, blockIndexes);
-  }
-
-  @Override
-  public DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndexes)
-      throws IOException {
-    DimensionColumnChunkReader dimensionChunksReader = getDimensionColumnChunkReader(fileReader);
-    return dimensionChunksReader.readRawDimensionChunk(fileReader, blockIndexes);
-  }
-
-  @Override
-  public MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, int[][] blockIndexes)
-      throws IOException {
-    MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
-    MeasureRawColumnChunk[] measureRawColumnChunks =
-        measureColumnChunkReader.readRawMeasureChunks(fileReader, blockIndexes);
-    updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunks);
-    return measureRawColumnChunks;
-  }
-
-  @Override public MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex)
-      throws IOException {
-    MeasureColumnChunkReader measureColumnChunkReader = getMeasureColumnChunkReader(fileReader);
-    MeasureRawColumnChunk measureRawColumnChunk =
-        measureColumnChunkReader.readRawMeasureChunk(fileReader, blockIndex);
-    updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunk);
-    return measureRawColumnChunk;
-  }
-
-  /**
-   * This method is written specifically for old store wherein the measure min and max values
-   * are written opposite (i.e min in place of max and amx in place of min). Due to this computing
-   * f measure filter with current code is impacted. In order to sync with current min and
-   * max values only in case old store and measures is reversed
-   *
-   * @param measureRawColumnChunk
-   */
-  private void updateMeasureRawColumnChunkMinMaxValues(
-      MeasureRawColumnChunk measureRawColumnChunk) {
-    if (blockInfos.get(index).isDataBlockFromOldStore()) {
-      byte[][] maxValues = measureRawColumnChunk.getMaxValues();
-      byte[][] minValues = measureRawColumnChunk.getMinValues();
-      measureRawColumnChunk.setMaxValues(minValues);
-      measureRawColumnChunk.setMinValues(maxValues);
-    }
-  }
-
-  private void updateMeasureRawColumnChunkMinMaxValues(
-      MeasureRawColumnChunk[] measureRawColumnChunks) {
-    if (blockInfos.get(index).isDataBlockFromOldStore()) {
-      for (int i = 0; i < measureRawColumnChunks.length; i++) {
-        if (null != measureRawColumnChunks[i]) {
-          updateMeasureRawColumnChunkMinMaxValues(measureRawColumnChunks[i]);
-        }
-      }
-    }
-  }
-
-  private DimensionColumnChunkReader getDimensionColumnChunkReader(FileHolder fileReader) {
-    ColumnarFormatVersion version =
-        ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
-    if (fileReader.isReadPageByPage()) {
-      return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
-          blockInfos.get(index).getFilePath(), true);
-    } else {
-      return CarbonDataReaderFactory.getInstance().getDimensionColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(), dimensionLens,
-          blockInfos.get(index).getFilePath(), false);
-    }
-  }
-
-  private MeasureColumnChunkReader getMeasureColumnChunkReader(FileHolder fileReader) {
-    ColumnarFormatVersion version =
-        ColumnarFormatVersion.valueOf(blockInfos.get(index).getDetailInfo().getVersionNumber());
-    if (fileReader.isReadPageByPage()) {
-      return CarbonDataReaderFactory.getInstance().getMeasureColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
-          blockInfos.get(index).getFilePath(), true);
-    } else {
-      return CarbonDataReaderFactory.getInstance().getMeasureColumnChunkReader(version,
-          blockInfos.get(index).getDetailInfo().getBlockletInfo(),
-          blockInfos.get(index).getFilePath(), false);
-    }
-  }
-
-  @Override
-  public void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache) {
-    this.deleteDeltaDataCache = deleteDeltaDataCache;
-  }
-
-  @Override public BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache() {
-    return deleteDeltaDataCache;
-  }
-
-  @Override public int numberOfPages() {
-    return blockInfos.get(index).getDetailInfo().getPagesCount();
-  }
-
-  @Override public int getPageRowCount(int pageNumber) {
-    return blockInfos.get(index).getDetailInfo().getBlockletInfo()
-        .getNumberOfRowsPerPage()[pageNumber];
-  }
-
-  public int numberOfNodes() {
-    return blockInfos.size();
-  }
-
-  public List<TableBlockInfo> getBlockInfos() {
-    return blockInfos;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
index 17ad17f..a30f64c 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
@@ -32,7 +32,7 @@ public class IndexWrapper extends AbstractIndex {
   public IndexWrapper(List<TableBlockInfo> blockInfos) {
     segmentProperties = new SegmentProperties(blockInfos.get(0).getDetailInfo().getColumnSchemas(),
         blockInfos.get(0).getDetailInfo().getDimLens());
-    dataRefNode = new BlockletDataRefNodeWrapper(blockInfos, 0,
+    dataRefNode = new BlockletDataRefNode(blockInfos, 0,
         segmentProperties.getDimensionColumnsValueSize());
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
index b88c1f4..537e124 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
@@ -19,7 +19,12 @@ package org.apache.carbondata.core.indexstore.blockletindex;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index 5862933..2203b3b 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -17,11 +17,11 @@
 
 package org.apache.carbondata.core.memory;
 
+import javax.annotation.concurrent.GuardedBy;
 import java.lang.ref.WeakReference;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
-import javax.annotation.concurrent.GuardedBy;
 
 /**
  * Code ported from Apache Spark {org.apache.spark.unsafe.memory} package


[13/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
index 9391ebd..44f7c07 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
@@ -16,20 +16,10 @@
  */
 package org.apache.carbondata.core.scan.filter.executer;
 
-import java.util.ArrayList;
 import java.util.BitSet;
-import java.util.List;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.executor.util.QueryUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.util.ByteUtil;
 
 /**
  * It checks if filter is required on given block and if required, it does
@@ -38,12 +28,6 @@ import org.apache.carbondata.core.util.ByteUtil;
 public class ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl {
 
   /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ExcludeColGroupFilterExecuterImpl.class.getName());
-
-  /**
    * @param dimColResolvedFilterInfo
    * @param segmentProperties
    */
@@ -53,54 +37,6 @@ public class ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl
   }
 
   /**
-   * 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.getExcludeFilterKeys();
-      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);
-    KeyStructureInfo restructureInfos = new KeyStructureInfo();
-    restructureInfos.setKeyGenerator(keyGenerator);
-    restructureInfos.setMaskByteRanges(maskByteRanges);
-    restructureInfos.setMaxKey(maxKey);
-    return restructureInfos;
-  }
-
-  /**
    * Check if scan is required on given block based on min and max value
    */
   public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
@@ -109,25 +45,4 @@ public class ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl
     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/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
index fad37fc..fe983a5 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -30,7 +30,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -40,13 +40,13 @@ import org.apache.carbondata.core.util.comparator.SerializableComparator;
 
 public class ExcludeFilterExecuterImpl implements FilterExecuter {
 
-  protected DimColumnResolvedFilterInfo dimColEvaluatorInfo;
-  protected DimColumnExecuterFilterInfo dimColumnExecuterInfo;
-  protected MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo;
-  protected MeasureColumnExecuterFilterInfo msrColumnExecutorInfo;
+  private DimColumnResolvedFilterInfo dimColEvaluatorInfo;
+  private DimColumnExecuterFilterInfo dimColumnExecuterInfo;
+  private MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo;
+  private MeasureColumnExecuterFilterInfo msrColumnExecutorInfo;
   protected SegmentProperties segmentProperties;
-  protected boolean isDimensionPresentInCurrentBlock = false;
-  protected boolean isMeasurePresentInCurrentBlock = false;
+  private boolean isDimensionPresentInCurrentBlock = false;
+  private boolean isMeasurePresentInCurrentBlock = false;
   private SerializableComparator comparator;
   /**
    * is dimension column data is natural sorted
@@ -82,48 +82,50 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     if (isDimensionPresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk dimensionRawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
-      DimensionColumnDataChunk[] dimensionColumnDataChunks =
-          dimensionRawColumnChunk.convertToDimColDataChunks();
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
+      DimensionColumnPage[] dimensionColumnPages =
+          dimensionRawColumnChunk.decodeAllColumnPages();
       BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount());
-      for (int i = 0; i < dimensionColumnDataChunks.length; i++) {
-        BitSet bitSet = getFilteredIndexes(dimensionColumnDataChunks[i],
+      for (int i = 0; i < dimensionColumnPages.length; i++) {
+        BitSet bitSet = getFilteredIndexes(dimensionColumnPages[i],
             dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-            blockChunkHolder.getBitSetGroup(), i);
+            rawBlockletColumnChunks.getBitSetGroup(), i);
         bitSetGroup.setBitSet(bitSet, i);
       }
 
       return bitSetGroup;
     } else if (isMeasurePresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       MeasureRawColumnChunk measureRawColumnChunk =
-          blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       ColumnPage[] ColumnPages =
-          measureRawColumnChunk.convertToColumnPage();
+          measureRawColumnChunk.decodeAllColumnPages();
       BitSetGroup bitSetGroup = new BitSetGroup(measureRawColumnChunk.getPagesCount());
       DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo);
       for (int i = 0; i < ColumnPages.length; i++) {
         BitSet bitSet =
             getFilteredIndexesForMeasure(
-                measureRawColumnChunk.convertToColumnPage(i),
+                measureRawColumnChunk.decodeColumnPage(i),
                 measureRawColumnChunk.getRowCount()[i],
                 useBitsetPipeLine,
-                blockChunkHolder.getBitSetGroup(),
+                rawBlockletColumnChunks.getBitSetGroup(),
                 i,
                 msrType);
         bitSetGroup.setBitSet(bitSet, i);
@@ -133,7 +135,8 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     return null;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
     if (isDimensionPresentInCurrentBlock) {
       byte[][] filterValues = dimColumnExecuterInfo.getExcludeFilterKeys();
       byte[] col = (byte[])value.getVal(dimColEvaluatorInfo.getDimension().getOrdinal());
@@ -275,43 +278,41 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
 
   /**
    * Below method will be used to apply filter on dimension column
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numberOfRows
    * @param useBitsetPipeLine
    * @param prvBitSetGroup
    * @param pageNumber
    * @return filtered indexes bitset
    */
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numberOfRows, boolean useBitsetPipeLine, BitSetGroup prvBitSetGroup, int pageNumber) {
     // check whether applying filtered based on previous bitset will be optimal
     if (CarbonUtil.usePreviousFilterBitsetGroup(useBitsetPipeLine, prvBitSetGroup, pageNumber,
         dimColumnExecuterInfo.getExcludeFilterKeys().length)) {
-      return getFilteredIndexesUisngPrvBitset(dimensionColumnDataChunk, prvBitSetGroup, pageNumber,
-          numberOfRows);
+      return getFilteredIndexesUisngPrvBitset(dimensionColumnPage, prvBitSetGroup, pageNumber);
     } else {
-      return getFilteredIndexes(dimensionColumnDataChunk, numberOfRows);
+      return getFilteredIndexes(dimensionColumnPage, numberOfRows);
     }
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numberOfRows) {
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numberOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numberOfRows);
     }
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numberOfRows);
+    return setFilterdIndexToBitSet(dimensionColumnPage, numberOfRows);
   }
 
   /**
    * Below method will be used to apply filter based on previous filtered bitset
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param prvBitSetGroup
    * @param pageNumber
-   * @param numberOfRows
    * @return filtered indexes bitset
    */
-  private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensionColumnDataChunk,
-      BitSetGroup prvBitSetGroup, int pageNumber, int numberOfRows) {
+  private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnPage dimensionColumnPage,
+      BitSetGroup prvBitSetGroup, int pageNumber) {
     BitSet prvPageBitSet = prvBitSetGroup.getBitSet(pageNumber);
     BitSet bitSet = new BitSet();
     bitSet.or(prvPageBitSet);
@@ -319,11 +320,11 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     int compareResult = 0;
     // if dimension data was natural sorted then get the index from previous bitset
     // and use the same in next column data, otherwise use the inverted index reverse
-    if (!dimensionColumnDataChunk.isExplicitSorted()) {
+    if (!dimensionColumnPage.isExplicitSorted()) {
       for (int index = prvPageBitSet.nextSetBit(0);
            index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) {
         compareResult = CarbonUtil
-            .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1, index);
+            .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1, index);
         if (compareResult != 0) {
           bitSet.set(index);
         } else {
@@ -336,8 +337,8 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
       for (int index = prvPageBitSet.nextSetBit(0);
            index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) {
         compareResult = CarbonUtil
-            .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1,
-                dimensionColumnDataChunk.getInvertedReverseIndex(index));
+            .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1,
+                dimensionColumnPage.getInvertedReverseIndex(index));
         if (compareResult != 0) {
           bitSet.set(index);
         } else {
@@ -351,7 +352,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
   }
 
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     bitSet.flip(0, numerOfRows);
     int startIndex = 0;
@@ -361,10 +362,10 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
         break;
       }
       int[] rangeIndex = CarbonUtil
-          .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i]);
       for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) {
-        bitSet.flip(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.flip(dimensionColumnPage.getInvertedIndex(j));
       }
       if (rangeIndex[1] >= 0) {
         startIndex = rangeIndex[1] + 1;
@@ -373,7 +374,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
 
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     bitSet.flip(0, numerOfRows);
@@ -390,7 +391,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
           break;
         }
         int[] rangeIndex = CarbonUtil
-            .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
                 filterValues[i]);
         for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) {
           bitSet.flip(j);
@@ -403,14 +404,14 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
       if (filterValues.length > 1) {
         for (int i = 0; i < numerOfRows; i++) {
           int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1,
-              dimensionColumnDataChunk.getChunkData(i));
+              dimensionColumnPage.getChunkData(i));
           if (index >= 0) {
             bitSet.flip(i);
           }
         }
       } else {
         for (int j = 0; j < numerOfRows; j++) {
-          if (dimensionColumnDataChunk.compareTo(j, filterValues[0]) == 0) {
+          if (dimensionColumnPage.compareTo(j, filterValues[0]) == 0) {
             bitSet.flip(j);
           }
         }
@@ -419,26 +420,30 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     bitSet.flip(0, 1);
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     if (isDimensionPresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
index 53d3068..a64341e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FilterExecuter.java
@@ -21,7 +21,7 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 public interface FilterExecuter {
@@ -32,8 +32,8 @@ public interface FilterExecuter {
    * @return
    * @throws FilterUnsupportedException
    */
-  BitSetGroup applyFilter(BlocksChunkHolder blocksChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException;
+  BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException;
 
   /**
    * apply range filter on a row
@@ -54,7 +54,7 @@ public interface FilterExecuter {
 
   /**
    * It just reads necessary block for filter executor, it does not uncompress the data.
-   * @param blockChunkHolder
+   * @param rawBlockletColumnChunks
    */
-  void readBlocks(BlocksChunkHolder blockChunkHolder)throws IOException;
+  void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks)throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java
index 9c4c7ba..b683fd6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ImplicitIncludeFilterExecutorImpl.java
@@ -23,7 +23,7 @@ import java.util.BitSet;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
@@ -41,26 +41,31 @@ public class ImplicitIncludeFilterExecutorImpl
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeline)
-      throws FilterUnsupportedException {
-    BitSetGroup bitSetGroup = new BitSetGroup(blockChunkHolder.getDataBlock().numberOfPages());
-    for (int i = 0; i < blockChunkHolder.getDataBlock().numberOfPages(); i++) {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeline) {
+    BitSetGroup bitSetGroup = new BitSetGroup(
+        rawBlockletColumnChunks.getDataBlock().numberOfPages());
+    for (int i = 0; i < rawBlockletColumnChunks.getDataBlock().numberOfPages(); i++) {
       bitSetGroup.setBitSet(
-          setBitSetForCompleteDimensionData(blockChunkHolder.getDataBlock().getPageRowCount(i)), i);
+          setBitSetForCompleteDimensionData(
+              rawBlockletColumnChunks.getDataBlock().getPageRowCount(i)), i);
     }
     return bitSetGroup;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException, IOException {
     return false;
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     return null;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
 
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
index 1cbc0bc..e4da26f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
@@ -24,14 +24,14 @@ import java.util.List;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
 import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
 import org.apache.carbondata.core.scan.executor.util.QueryUtil;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 
@@ -59,7 +59,7 @@ public class IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl
   /**
    * It fills BitSet with row index which matches filter key
    */
-  protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  protected BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
 
@@ -70,7 +70,7 @@ public class IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl
         byte[] filterVal = filterValues[i];
         for (int rowId = 0; rowId < numerOfRows; rowId++) {
           byte[] colData = new byte[keyStructureInfo.getMaskByteRanges().length];
-          dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, keyStructureInfo);
+          dimensionColumnPage.fillRawData(rowId, 0, colData, keyStructureInfo);
           if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) == 0) {
             bitSet.set(rowId);
           }
@@ -85,20 +85,21 @@ public class IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
+    int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
         .get(dimColumnEvaluatorInfo.getColumnIndex());
-    if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-      blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+      rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+          rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+              rawBlockletColumnChunks.getFileReader(), chunkIndex);
     }
     DimensionRawColumnChunk dimensionRawColumnChunk =
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
     BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount());
     for (int i = 0; i < dimensionRawColumnChunk.getPagesCount(); i++) {
       if (dimensionRawColumnChunk.getMaxValues() != null) {
-        BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i),
+        BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i),
             dimensionRawColumnChunk.getRowCount()[i]);
         bitSetGroup.setBitSet(bitSet, i);
       }
@@ -133,10 +134,10 @@ public class IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl
     BitSet bitSet = new BitSet(1);
     byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
     int columnIndex = dimColumnEvaluatorInfo.getColumnIndex();
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex);
+    int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping().get(columnIndex);
     int[] cols = getAllColumns(columnIndex);
-    byte[] maxValue = getMinMaxData(cols, blkMaxVal[blockIndex], columnIndex);
-    byte[] minValue = getMinMaxData(cols, blkMinVal[blockIndex], columnIndex);
+    byte[] maxValue = getMinMaxData(cols, blkMaxVal[chunkIndex], columnIndex);
+    byte[] minValue = getMinMaxData(cols, blkMinVal[chunkIndex], 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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
index fe1421c..05328f3 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -30,7 +30,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -41,12 +41,12 @@ import org.apache.carbondata.core.util.comparator.SerializableComparator;
 public class IncludeFilterExecuterImpl implements FilterExecuter {
 
   protected DimColumnResolvedFilterInfo dimColumnEvaluatorInfo;
-  protected DimColumnExecuterFilterInfo dimColumnExecuterInfo;
-  protected MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo;
-  protected MeasureColumnExecuterFilterInfo msrColumnExecutorInfo;
+  DimColumnExecuterFilterInfo dimColumnExecuterInfo;
+  private MeasureColumnResolvedFilterInfo msrColumnEvaluatorInfo;
+  private MeasureColumnExecuterFilterInfo msrColumnExecutorInfo;
   protected SegmentProperties segmentProperties;
-  protected boolean isDimensionPresentInCurrentBlock = false;
-  protected boolean isMeasurePresentInCurrentBlock = false;
+  private boolean isDimensionPresentInCurrentBlock = false;
+  private boolean isMeasurePresentInCurrentBlock = false;
   protected SerializableComparator comparator;
   /**
    * is dimension column data is natural sorted
@@ -86,44 +86,46 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
-    if (isDimensionPresentInCurrentBlock == true) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
+    if (isDimensionPresentInCurrentBlock) {
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk dimensionRawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(dimensionRawColumnChunk.getPagesCount());
       for (int i = 0; i < dimensionRawColumnChunk.getPagesCount(); i++) {
         if (dimensionRawColumnChunk.getMaxValues() != null) {
           if (isScanRequired(dimensionRawColumnChunk.getMaxValues()[i],
               dimensionRawColumnChunk.getMinValues()[i], dimColumnExecuterInfo.getFilterKeys())) {
-            BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i),
+            BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i),
                 dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-                blockChunkHolder.getBitSetGroup(), i);
+                rawBlockletColumnChunks.getBitSetGroup(), i);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(dimensionRawColumnChunk.decodeColumnPage(i),
               dimensionRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-              blockChunkHolder.getBitSetGroup(), i);
+              rawBlockletColumnChunks.getBitSetGroup(), i);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     } else if (isMeasurePresentInCurrentBlock) {
-      int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
+      int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       MeasureRawColumnChunk measureRawColumnChunk =
-          blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(measureRawColumnChunk.getPagesCount());
       DataType msrType = getMeasureDataType(msrColumnEvaluatorInfo);
       for (int i = 0; i < measureRawColumnChunk.getPagesCount(); i++) {
@@ -132,16 +134,16 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
               measureRawColumnChunk.getMinValues()[i], msrColumnExecutorInfo.getFilterKeys(),
               msrColumnEvaluatorInfo.getType())) {
             BitSet bitSet =
-                getFilteredIndexesForMeasure(measureRawColumnChunk.convertToColumnPage(i),
+                getFilteredIndexesForMeasure(measureRawColumnChunk.decodeColumnPage(i),
                     measureRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-                    blockChunkHolder.getBitSetGroup(), i, msrType);
+                    rawBlockletColumnChunks.getBitSetGroup(), i, msrType);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasure(measureRawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasure(measureRawColumnChunk.decodeColumnPage(i),
                   measureRawColumnChunk.getRowCount()[i], useBitsetPipeLine,
-                  blockChunkHolder.getBitSetGroup(), i, msrType);
+                  rawBlockletColumnChunks.getBitSetGroup(), i, msrType);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
@@ -293,43 +295,43 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
 
   /**
    * Below method will be used to apply filter on dimension column
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numberOfRows
    * @param useBitsetPipeLine
    * @param prvBitSetGroup
    * @param pageNumber
    * @return filtered indexes bitset
    */
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numberOfRows, boolean useBitsetPipeLine, BitSetGroup prvBitSetGroup, int pageNumber) {
     // check whether previous indexes can be optimal to apply filter on dimension column
     if (CarbonUtil.usePreviousFilterBitsetGroup(useBitsetPipeLine, prvBitSetGroup, pageNumber,
         dimColumnExecuterInfo.getFilterKeys().length)) {
-      return getFilteredIndexesUisngPrvBitset(dimensionColumnDataChunk, prvBitSetGroup, pageNumber,
+      return getFilteredIndexesUisngPrvBitset(dimensionColumnPage, prvBitSetGroup, pageNumber,
           numberOfRows);
     } else {
-      return getFilteredIndexes(dimensionColumnDataChunk, numberOfRows);
+      return getFilteredIndexes(dimensionColumnPage, numberOfRows);
     }
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numberOfRows) {
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numberOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numberOfRows);
     }
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numberOfRows);
+    return setFilterdIndexToBitSet(dimensionColumnPage, numberOfRows);
   }
 
   /**
    * Below method will be used to apply filter on dimension
    * column based on previous filtered indexes
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param prvBitSetGroup
    * @param pageNumber
    * @param numberOfRows
    * @return filtered bitset
    */
-  private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnPage dimensionColumnPage,
       BitSetGroup prvBitSetGroup, int pageNumber, int numberOfRows) {
     BitSet prvPageBitSet = prvBitSetGroup.getBitSet(pageNumber);
     BitSet bitSet = new BitSet(numberOfRows);
@@ -337,11 +339,11 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     int compareResult = 0;
     // if dimension data was natural sorted then get the index from previous bitset
     // and use the same in next column data, otherwise use the inverted index reverse
-    if (!dimensionColumnDataChunk.isExplicitSorted()) {
+    if (!dimensionColumnPage.isExplicitSorted()) {
       for (int index = prvPageBitSet.nextSetBit(0);
            index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) {
         compareResult = CarbonUtil
-            .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1, index);
+            .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1, index);
         if (compareResult == 0) {
           bitSet.set(index);
         }
@@ -350,8 +352,8 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
       for (int index = prvPageBitSet.nextSetBit(0);
            index >= 0; index = prvPageBitSet.nextSetBit(index + 1)) {
         compareResult = CarbonUtil
-            .isFilterPresent(filterKeys, dimensionColumnDataChunk, 0, filterKeys.length - 1,
-                dimensionColumnDataChunk.getInvertedReverseIndex(index));
+            .isFilterPresent(filterKeys, dimensionColumnPage, 0, filterKeys.length - 1,
+                dimensionColumnPage.getInvertedReverseIndex(index));
         if (compareResult == 0) {
           bitSet.set(index);
         }
@@ -360,7 +362,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     int startIndex = 0;
     byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
@@ -369,10 +371,10 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
         break;
       }
       int[] rangeIndex = CarbonUtil
-          .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i]);
       for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
       }
       if (rangeIndex[1] >= 0) {
         startIndex = rangeIndex[1] + 1;
@@ -381,7 +383,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
 
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
@@ -394,7 +396,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
           break;
         }
         int[] rangeIndex = CarbonUtil
-            .getRangeIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            .getRangeIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
                 filterValues[i]);
         for (int j = rangeIndex[0]; j <= rangeIndex[1]; j++) {
           bitSet.set(j);
@@ -407,14 +409,14 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
       if (filterValues.length > 1) {
         for (int i = 0; i < numerOfRows; i++) {
           int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1,
-              dimensionColumnDataChunk.getChunkData(i));
+              dimensionColumnPage.getChunkData(i));
           if (index >= 0) {
             bitSet.set(i);
           }
         }
       } else {
         for (int j = 0; j < numerOfRows; j++) {
-          if (dimensionColumnDataChunk.compareTo(j, filterValues[0]) == 0) {
+          if (dimensionColumnPage.compareTo(j, filterValues[0]) == 0) {
             bitSet.set(j);
           }
         }
@@ -423,26 +425,25 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return bitSet;
   }
 
+  @Override
   public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
     BitSet bitSet = new BitSet(1);
     byte[][] filterValues = null;
     int columnIndex = 0;
-    int blockIndex = 0;
+    int chunkIndex = 0;
     boolean isScanRequired = false;
 
     if (isDimensionPresentInCurrentBlock) {
       filterValues = dimColumnExecuterInfo.getFilterKeys();
       columnIndex = dimColumnEvaluatorInfo.getColumnIndex();
-      blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex);
-      isScanRequired =
-          isScanRequired(blkMaxVal[blockIndex], blkMinVal[blockIndex], filterValues);
-
+      chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping().get(columnIndex);
+      isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex], filterValues);
     } else if (isMeasurePresentInCurrentBlock) {
       columnIndex = msrColumnEvaluatorInfo.getColumnIndex();
-      blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(columnIndex) + segmentProperties
-              .getLastDimensionColOrdinal();
-      isScanRequired = isScanRequired(blkMaxVal[blockIndex], blkMinVal[blockIndex],
+      chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(columnIndex) +
+              segmentProperties.getLastDimensionColOrdinal();
+      isScanRequired = isScanRequired(blkMaxVal[chunkIndex], blkMinVal[chunkIndex],
           msrColumnExecutorInfo.getFilterKeys(),
           msrColumnEvaluatorInfo.getType());
     }
@@ -492,20 +493,23 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     return false;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
-    if (isDimensionPresentInCurrentBlock == true) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    if (isDimensionPresentInCurrentBlock) {
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-    } else if (isMeasurePresentInCurrentBlock == true) {
-      int blockIndex = segmentProperties.getMeasuresOrdinalToBlockMapping()
+    } else if (isMeasurePresentInCurrentBlock) {
+      int chunkIndex = segmentProperties.getMeasuresOrdinalToChunkMapping()
           .get(msrColumnEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
index 87273bb..9f0afb5 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/OrFilterExecuterImpl.java
@@ -21,7 +21,7 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 public class OrFilterExecuterImpl implements FilterExecuter {
@@ -35,30 +35,33 @@ public class OrFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    BitSetGroup leftFilters = leftExecuter.applyFilter(blockChunkHolder, false);
-    BitSetGroup rightFilters = rightExecuter.applyFilter(blockChunkHolder, false);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    BitSetGroup leftFilters = leftExecuter.applyFilter(rawBlockletColumnChunks, false);
+    BitSetGroup rightFilters = rightExecuter.applyFilter(rawBlockletColumnChunks, false);
     leftFilters.or(rightFilters);
-    blockChunkHolder.setBitSetGroup(leftFilters);
+    rawBlockletColumnChunks.setBitSetGroup(leftFilters);
     return leftFilters;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException, IOException {
     return leftExecuter.applyFilter(value, dimOrdinalMax) ||
         rightExecuter.applyFilter(value, dimOrdinalMax);
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue);
     BitSet rightFilters = rightExecuter.isScanRequired(blockMaxValue, blockMinValue);
     leftFilters.or(rightFilters);
     return leftFilters;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
-    leftExecuter.readBlocks(blockChunkHolder);
-    rightExecuter.readBlocks(blockChunkHolder);
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    leftExecuter.readColumnChunks(rawBlockletColumnChunks);
+    rightExecuter.readColumnChunks(rawBlockletColumnChunks);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
index 797fe9d..34555e1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RangeValueFilterExecuterImpl.java
@@ -22,11 +22,10 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -39,8 +38,7 @@ import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedExc
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -60,25 +58,23 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
   /**
    * it has index at which given dimension is stored in file
    */
-  protected int dimensionBlocksIndex;
+  private int dimensionChunkIndex;
 
   /**
    * flag to check whether the filter dimension is present in current block list of dimensions.
    * Applicable for restructure scenarios
    */
-  protected boolean isDimensionPresentInCurrentBlock;
-  boolean lessThanExp;
-  boolean lessThanEqualExp;
-  boolean greaterThanExp;
-  boolean greaterThanEqualExp;
-  boolean startBlockMinIsDefaultStart;
-  boolean endBlockMaxisDefaultEnd;
-  boolean isRangeFullyCoverBlock;
+  private boolean isDimensionPresentInCurrentBlock;
+  private boolean lessThanExp;
+  private boolean lessThanEqualExp;
+  private boolean greaterThanExp;
+  private boolean greaterThanEqualExp;
+  private boolean startBlockMinIsDefaultStart;
+  private boolean endBlockMaxisDefaultEnd;
+  private boolean isRangeFullyCoverBlock;
 
   public RangeValueFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo,
-      MeasureColumnResolvedFilterInfo msrColEvaluatorInfo, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
-      SegmentProperties segmentProperties) {
+      Expression exp, byte[][] filterRangeValues, SegmentProperties segmentProperties) {
 
     this.dimColEvaluatorInfo = dimColEvaluatorInfo;
     this.exp = exp;
@@ -91,7 +87,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     startBlockMinIsDefaultStart = false;
     endBlockMaxisDefaultEnd = false;
     isRangeFullyCoverBlock = false;
-    initDimensionBlockIndexes();
+    initDimensionChunkIndexes();
     ifDefaultValueMatchesFilter();
 
   }
@@ -100,13 +96,13 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * This method will initialize the dimension info for the current block to be
    * used for filtering the data
    */
-  private void initDimensionBlockIndexes() {
+  private void initDimensionChunkIndexes() {
     // find the dimension in the current block dimensions list
     CarbonDimension dimensionFromCurrentBlock =
         segmentProperties.getDimensionFromCurrentBlock(dimColEvaluatorInfo.getDimension());
     if (null != dimensionFromCurrentBlock) {
       dimColEvaluatorInfo.setColumnIndex(dimensionFromCurrentBlock.getOrdinal());
-      this.dimensionBlocksIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+      this.dimensionChunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimensionFromCurrentBlock.getOrdinal());
       isDimensionPresentInCurrentBlock = true;
     }
@@ -136,14 +132,14 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
 
   /**
    * Method to apply the filter.
-   * @param blockChunkHolder
+   * @param rawBlockletColumnChunks
    * @return
    * @throws FilterUnsupportedException
    * @throws IOException
    */
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    return applyNoAndDirectFilter(blockChunkHolder);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    return applyNoAndDirectFilter(rawBlockletColumnChunks);
   }
 
   /**
@@ -272,32 +268,32 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     //         Filter Min <-----------------------------------------------> Filter Max
 
     if (isDimensionPresentInCurrentBlock) {
-      if (((lessThanExp == true) && (
+      if (((lessThanExp) && (
           ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[1]) >= 0)) || (
-          (lessThanEqualExp == true) && (
+          (lessThanEqualExp) && (
               ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[1]) > 0)) || (
-          (greaterThanExp == true) && (
+          (greaterThanExp) && (
               ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[0], blockMaxValue) >= 0)) || (
-          (greaterThanEqualExp == true) && (
+          (greaterThanEqualExp) && (
               ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[0], blockMaxValue) > 0))) {
         // completely out of block boundary
         isScanRequired = false;
       } else {
-        if (((greaterThanExp == true) && (
+        if (((greaterThanExp) && (
             ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[0]) > 0)) || (
-            (greaterThanEqualExp == true) && (
+            (greaterThanEqualExp) && (
                 ByteUtil.UnsafeComparer.INSTANCE.compareTo(blockMinValue, filterValues[0]) >= 0))) {
           startBlockMinIsDefaultStart = true;
         }
 
-        if (((lessThanExp == true) && (
+        if (((lessThanExp) && (
             ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[1], blockMaxValue) > 0)) || (
-            (lessThanEqualExp == true) && (
+            (lessThanEqualExp) && (
                 ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[1], blockMaxValue) >= 0))) {
           endBlockMaxisDefaultEnd = true;
         }
 
-        if (startBlockMinIsDefaultStart == true && endBlockMaxisDefaultEnd == true) {
+        if (startBlockMinIsDefaultStart && endBlockMaxisDefaultEnd) {
           isRangeFullyCoverBlock = true;
         }
       }
@@ -313,7 +309,8 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * @param blockMinValue
    * @return
    */
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     byte[][] filterValues = this.filterRangesValues;
     int columnIndex = this.dimColEvaluatorInfo.getColumnIndex();
@@ -329,50 +326,49 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * Method to apply the Range Filter.
    * @param blockChunkHolder
    * @return
-   * @throws FilterUnsupportedException
    * @throws IOException
    */
-  public BitSetGroup applyNoAndDirectFilter(BlocksChunkHolder blockChunkHolder)
-      throws FilterUnsupportedException, IOException {
+  private BitSetGroup applyNoAndDirectFilter(RawBlockletColumnChunks blockChunkHolder)
+      throws IOException {
 
     // In case of Alter Table Add and Delete Columns the isDimensionPresentInCurrentBlock can be
     // false, in that scenario the default values of the column should be shown.
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
-      return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
-              numberOfRows, true);
+      int numberOfRows = blockChunkHolder.getDataBlock().numRows();
+      return FilterUtil.createBitSetGroupWithDefaultValue(
+          blockChunkHolder.getDataBlock().numberOfPages(), numberOfRows, true);
     }
 
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+    int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
         .get(dimColEvaluatorInfo.getColumnIndex());
 
-    if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-      blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    if (null == blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex]) {
+      blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex] =
+          blockChunkHolder.getDataBlock().readDimensionChunk(
+              blockChunkHolder.getFileReader(), chunkIndex);
     }
 
     DimensionRawColumnChunk rawColumnChunk =
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+        blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex];
     BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
     for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
       if (rawColumnChunk.getMaxValues() != null) {
         if (isScanRequired(rawColumnChunk.getMinValues()[i], rawColumnChunk.getMaxValues()[i],
             this.filterRangesValues)) {
-          if (isRangeFullyCoverBlock == true) {
+          if (isRangeFullyCoverBlock) {
             // Set all the bits in this case as filter Min Max values cover the whole block.
             BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]);
             bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           } else {
-            BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+            BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                 rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         }
       } else {
-        BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+        BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
             rawColumnChunk.getRowCount()[i]);
         bitSetGroup.setBitSet(bitSet, i);
       }
@@ -380,12 +376,12 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     return bitSetGroup;
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      return setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows);
     }
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+    return setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows);
   }
 
   /**
@@ -394,12 +390,12 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
     int startIndex = 0;
@@ -410,13 +406,13 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     // For Range expression we expect two values. The First is the Min Value and Second is the
     // Max value.
     // Get the Min Value
-    if (startBlockMinIsDefaultStart == false) {
+    if (!startBlockMinIsDefaultStart) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[0], greaterThanExp);
-      if (greaterThanExp == true && start >= 0) {
+      if (greaterThanExp && start >= 0) {
         start = CarbonUtil
-            .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0],
+            .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[0],
                 numerOfRows);
       }
 
@@ -428,8 +424,8 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
         // 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[0], dimensionColumnDataChunk
-            .getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))) > 0) {
+        if ((ByteUtil.compare(filterValues[0], dimensionColumnPage
+            .getChunkData(dimensionColumnPage.getInvertedIndex(start)))) > 0) {
           start = start + 1;
         }
       }
@@ -440,14 +436,14 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     }
 
     // Get the Max value
-    if (endBlockMaxisDefaultEnd == false) {
+    if (!endBlockMaxisDefaultEnd) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[1], lessThanEqualExp);
 
-      if (lessThanExp == true && start >= 0) {
+      if (lessThanExp && start >= 0) {
         start =
-            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]);
+            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[1]);
       }
 
       if (start < 0) {
@@ -458,7 +454,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
         // In case the start is less than 0, then positive value of start is pointing to the next
         // value of the searched key. So move to the previous one.
         if ((ByteUtil.compare(filterValues[1],
-            dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             < 0)) {
           start = start - 1;
         }
@@ -469,18 +465,18 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
     }
 
     for (int j = startMin; j <= endMax; j++) {
-      bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+      bitSet.set(dimensionColumnPage.getInvertedIndex(j));
     }
 
     // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on
     // matching row.
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet);
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      updateForNoDictionaryColumn(startMin, endMax, dimensionColumnPage, bitSet);
     }
     return bitSet;
   }
 
-  private void updateForNoDictionaryColumn(int start, int end, DimensionColumnDataChunk dataChunk,
+  private void updateForNoDictionaryColumn(int start, int end, DimensionColumnPage dataChunk,
       BitSet bitset) {
     for (int j = start; j <= end; j++) {
       if (dataChunk.compareTo(j, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY) == 0
@@ -496,31 +492,31 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
    * be called if the column is sorted default so column index
    * mapping will be present for accesaing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
-    // if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+    // if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) {
     byte[][] filterValues = this.filterRangesValues;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
+    if (dimensionColumnPage.isExplicitSorted()) {
       int start = 0;
       int startMin = 0;
       int endMax = 0;
       int startIndex = 0;
       // For Range expression we expect two values. The First is the Min Value and Second is the
       // Max value.
-      if (startBlockMinIsDefaultStart == false) {
+      if (!startBlockMinIsDefaultStart) {
 
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
                 filterValues[0], greaterThanExp);
 
-        if (greaterThanExp == true && start >= 0) {
+        if (greaterThanExp && start >= 0) {
           start = CarbonUtil
-              .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[0],
+              .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[0],
                   numerOfRows);
         }
 
@@ -532,7 +528,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
           // 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[0], dimensionColumnDataChunk.getChunkData(start)))
+          if ((ByteUtil.compare(filterValues[0], dimensionColumnPage.getChunkData(start)))
               > 0) {
             start = start + 1;
           }
@@ -542,14 +538,14 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
         startMin = startIndex;
       }
 
-      if (endBlockMaxisDefaultEnd == false) {
+      if (!endBlockMaxisDefaultEnd) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
                 filterValues[1], lessThanEqualExp);
 
-        if (lessThanExp == true && start >= 0) {
+        if (lessThanExp && start >= 0) {
           start =
-              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[1]);
+              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[1]);
         }
 
         if (start < 0) {
@@ -559,7 +555,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
           }
           // In case the start is less than 0, then positive value of start is pointing to the next
           // value of the searched key. So move to the previous one.
-          if ((ByteUtil.compare(filterValues[1], dimensionColumnDataChunk.getChunkData(start))
+          if ((ByteUtil.compare(filterValues[1], dimensionColumnPage.getChunkData(start))
               < 0)) {
             start = start - 1;
           }
@@ -575,8 +571,8 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
 
       // Binary Search cannot be done on '@NU#LL$!", so need to check and compare for null on
       // matching row.
-      if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-        updateForNoDictionaryColumn(startMin, endMax, dimensionColumnDataChunk, bitSet);
+      if (dimensionColumnPage.isNoDicitionaryColumn()) {
+        updateForNoDictionaryColumn(startMin, endMax, dimensionColumnPage, bitSet);
       }
     } else {
       byte[] defaultValue = null;
@@ -585,7 +581,7 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
             .getDirectDictionaryGenerator(dimColEvaluatorInfo.getDimension().getDataType());
         int key = directDictionaryGenerator.generateDirectSurrogateKey(null);
         CarbonDimension currentBlockDimension =
-            segmentProperties.getDimensions().get(dimensionBlocksIndex);
+            segmentProperties.getDimensions().get(dimensionChunkIndex);
         if (currentBlockDimension.isSortColumn()) {
           defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
               this.segmentProperties.getSortColumnsGenerator());
@@ -601,55 +597,37 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
       }
       // evaluate result for lower range value first and then perform and operation in the
       // upper range value in order to compute the final result
-      bitSet = evaluateGreaterThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[0],
+      bitSet = evaluateGreaterThanFilterForUnsortedColumn(dimensionColumnPage, filterValues[0],
           numerOfRows);
       BitSet upperRangeBitSet =
-          evaluateLessThanFilterForUnsortedColumn(dimensionColumnDataChunk, filterValues[1],
+          evaluateLessThanFilterForUnsortedColumn(dimensionColumnPage, filterValues[1],
               numerOfRows);
       bitSet.and(upperRangeBitSet);
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet, defaultValue);
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet, defaultValue);
     }
     return bitSet;
   }
 
   /**
-   * This method will compare the selected data against null values and
-   * flip the bitSet if any null value is found
-   *
-   * @param dimensionColumnDataChunk
-   * @param bitSet
-   */
-  private void removeNullValues(DimensionColumnDataChunk dimensionColumnDataChunk, BitSet bitSet) {
-    if (!bitSet.isEmpty()) {
-      for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-        if (dimensionColumnDataChunk.compareTo(i, CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY)
-            == 0) {
-          bitSet.flip(i);
-        }
-      }
-    }
-  }
-
-  /**
    * This method will evaluate the result for filter column based on the lower range value
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param filterValue
    * @param numberOfRows
    * @return
    */
   private BitSet evaluateGreaterThanFilterForUnsortedColumn(
-      DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) {
+      DimensionColumnPage dimensionColumnPage, byte[] filterValue, int numberOfRows) {
     BitSet bitSet = new BitSet(numberOfRows);
     if (greaterThanExp) {
       for (int i = 0; i < numberOfRows; i++) {
-        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) > 0)) {
+        if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) > 0)) {
           bitSet.set(i);
         }
       }
     } else if (greaterThanEqualExp) {
       for (int i = 0; i < numberOfRows; i++) {
-        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) >= 0)) {
+        if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) >= 0)) {
           bitSet.set(i);
         }
       }
@@ -660,23 +638,23 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
   /**
    * This method will evaluate the result for filter column based on the upper range value
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param filterValue
    * @param numberOfRows
    * @return
    */
   private BitSet evaluateLessThanFilterForUnsortedColumn(
-      DimensionColumnDataChunk dimensionColumnDataChunk, byte[] filterValue, int numberOfRows) {
+      DimensionColumnPage dimensionColumnPage, byte[] filterValue, int numberOfRows) {
     BitSet bitSet = new BitSet(numberOfRows);
     if (lessThanExp) {
       for (int i = 0; i < numberOfRows; i++) {
-        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) < 0)) {
+        if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) < 0)) {
           bitSet.set(i);
         }
       }
     } else if (lessThanEqualExp) {
       for (int i = 0; i < numberOfRows; i++) {
-        if ((ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValue) <= 0)) {
+        if ((ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValue) <= 0)) {
           bitSet.set(i);
         }
       }
@@ -686,16 +664,18 @@ public class RangeValueFilterExecuterImpl extends ValueBasedFilterExecuterImpl {
 
   /**
    * Method to read the blocks.
-   * @param blockChunkHolder
+   * @param rawBlockletColumnChunks
    * @throws IOException
    */
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
-    if (isDimensionPresentInCurrentBlock == true) {
-      int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    if (isDimensionPresentInCurrentBlock) {
+      int chunkIndex = segmentProperties.getDimensionOrdinalToChunkMapping()
           .get(dimColEvaluatorInfo.getColumnIndex());
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
index 5707eb4..d7bec7e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureExcludeFilterExecutorImpl.java
@@ -24,7 +24,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 public class RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorImpl {
@@ -34,7 +34,7 @@ public class RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorIm
    * flag to check whether filter values contain the default value applied on the dimension column
    * which does not exist in the current block
    */
-  protected boolean isDefaultValuePresentInFilterValues;
+  private boolean isDefaultValuePresentInFilterValues;
 
   public RestructureExcludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo,
       MeasureColumnResolvedFilterInfo measureColumnResolvedFilterInfo, boolean isMeasure) {
@@ -48,26 +48,29 @@ public class RestructureExcludeFilterExecutorImpl extends RestructureEvaluatorIm
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
-    int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
+    int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
     return FilterUtil
-        .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+        .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
             numberOfRows, !isDefaultValuePresentInFilterValues);
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException {
     throw new FilterUnsupportedException("Unsupported RestructureExcludeFilterExecutorImpl on row");
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     bitSet.flip(0, 1);
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) {
 
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
index 8bcc53f..c874fc5 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RestructureIncludeFilterExecutorImpl.java
@@ -24,7 +24,7 @@ import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorImpl {
@@ -33,7 +33,7 @@ public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorIm
    * flag to check whether filter values contain the default value applied on the dimension column
    * which does not exist in the current block
    */
-  protected boolean isDefaultValuePresentInFilterValues;
+  private boolean isDefaultValuePresentInFilterValues;
 
   public RestructureIncludeFilterExecutorImpl(DimColumnResolvedFilterInfo dimColumnEvaluatorInfo,
       MeasureColumnResolvedFilterInfo measureColumnResolvedFilterInfo, boolean isMeasure) {
@@ -47,15 +47,16 @@ public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorIm
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws IOException {
-    int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
-    return FilterUtil
-        .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
-            numberOfRows, isDefaultValuePresentInFilterValues);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
+    int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
+    return FilterUtil.createBitSetGroupWithDefaultValue(
+        rawBlockletColumnChunks.getDataBlock().numberOfPages(),
+        numberOfRows, isDefaultValuePresentInFilterValues);
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException {
     throw new FilterUnsupportedException("Unsupported RestructureIncludeFilterExecutorImpl on row");
   }
@@ -66,7 +67,8 @@ public class RestructureIncludeFilterExecutorImpl extends RestructureEvaluatorIm
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) {
 
   }
 


[34/49] carbondata git commit: Revert "[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row"

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 527452a..11b3d43 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -31,14 +31,15 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
@@ -62,15 +63,21 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    * entry count
    */
   private int entryCount;
+
   /**
    * return row
    */
-  private IntermediateSortTempRow returnRow;
+  private Object[] returnRow;
+  private int dimCnt;
+  private int complexCnt;
+  private int measureCnt;
+  private boolean[] isNoDictionaryDimensionColumn;
+  private DataType[] measureDataTypes;
   private int readBufferSize;
   private String compressorName;
-  private IntermediateSortTempRow[] currentBuffer;
+  private Object[][] currentBuffer;
 
-  private IntermediateSortTempRow[] backupBuffer;
+  private Object[][] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -93,21 +100,27 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
   private int numberOfObjectRead;
 
-  private TableFieldStat tableFieldStat;
-  private SortStepRowHandler sortStepRowHandler;
-  private Comparator<IntermediateSortTempRow> comparator;
+  private int nullSetWordsLength;
+
+  private Comparator<Object[]> comparator;
+
   /**
    * Constructor to initialize
    */
   public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters parameters) {
     // set temp file
     this.tempFile = tempFile;
+    this.dimCnt = parameters.getDimColCount();
+    this.complexCnt = parameters.getComplexDimColCount();
+    this.measureCnt = parameters.getMeasureColCount();
+    this.isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
+    this.measureDataTypes = parameters.getMeasureDataType();
     this.readBufferSize = parameters.getBufferSize();
     this.compressorName = parameters.getSortTempCompressorName();
-    this.tableFieldStat = new TableFieldStat(parameters);
-    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
+
     this.executorService = Executors.newFixedThreadPool(1);
-    comparator = new IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn());
+    this.nullSetWordsLength = ((parameters.getMeasureColCount() - 1) >> 6) + 1;
+    comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
     initialize();
   }
 
@@ -156,17 +169,11 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    *
    * @throws CarbonSortKeyAndGroupByException problem while reading
    */
-  @Override
   public void readRow() throws CarbonSortKeyAndGroupByException {
     if (prefetch) {
       fillDataForPrefetch();
     } else {
-      try {
-        this.returnRow = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
-        this.numberOfObjectRead++;
-      } catch (IOException e) {
-        throw new CarbonSortKeyAndGroupByException("Problems while reading row", e);
-      }
+      this.returnRow = getRowFromStream();
     }
   }
 
@@ -200,22 +207,63 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
   }
 
   /**
-   * get a batch of row, this interface is used in reading compressed sort temp files
-   *
-   * @param expected expected number in a batch
-   * @return a batch of row
-   * @throws IOException if error occurs while reading from stream
+   * @return
+   * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow[] readBatchedRowFromStream(int expected)
-      throws IOException {
-    IntermediateSortTempRow[] holders = new IntermediateSortTempRow[expected];
-    for (int i = 0; i < expected; i++) {
-      IntermediateSortTempRow holder
-          = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
-      holders[i] = holder;
+  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
+    Object[] row = new Object[dimCnt + measureCnt];
+    try {
+      int dimCount = 0;
+      for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
+        if (isNoDictionaryDimensionColumn[dimCount]) {
+          short aShort = stream.readShort();
+          byte[] col = new byte[aShort];
+          stream.readFully(col);
+          row[dimCount] = col;
+        } else {
+          int anInt = stream.readInt();
+          row[dimCount] = anInt;
+        }
+      }
+
+      // write complex dimensions here.
+      for (; dimCount < dimCnt; dimCount++) {
+        short aShort = stream.readShort();
+        byte[] col = new byte[aShort];
+        stream.readFully(col);
+        row[dimCount] = col;
+      }
+
+      long[] words = new long[nullSetWordsLength];
+      for (int i = 0; i < words.length; i++) {
+        words[i] = stream.readLong();
+      }
+
+      for (int mesCount = 0; mesCount < measureCnt; mesCount++) {
+        if (UnsafeCarbonRowPage.isSet(words, mesCount)) {
+          DataType dataType = measureDataTypes[mesCount];
+          if (dataType == DataTypes.SHORT) {
+            row[dimCount + mesCount] = stream.readShort();
+          } else if (dataType == DataTypes.INT) {
+            row[dimCount + mesCount] = stream.readInt();
+          } else if (dataType == DataTypes.LONG) {
+            row[dimCount + mesCount] = stream.readLong();
+          } else if (dataType == DataTypes.DOUBLE) {
+            row[dimCount + mesCount] = stream.readDouble();
+          } else if (DataTypes.isDecimal(dataType)) {
+            short aShort = stream.readShort();
+            byte[] bigDecimalInBytes = new byte[aShort];
+            stream.readFully(bigDecimalInBytes);
+            row[dimCount + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
+          } else {
+            throw new IllegalArgumentException("unsupported data type:" + dataType);
+          }
+        }
+      }
+      return row;
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException(e);
     }
-    this.numberOfObjectRead += expected;
-    return holders;
   }
 
   /**
@@ -223,7 +271,7 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    *
    * @return row
    */
-  public IntermediateSortTempRow getRow() {
+  public Object[] getRow() {
     return this.returnRow;
   }
 
@@ -278,7 +326,9 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
   @Override public int hashCode() {
     int hash = 0;
-    hash += tableFieldStat.hashCode();
+    hash += 31 * measureCnt;
+    hash += 31 * dimCnt;
+    hash += 31 * complexCnt;
     hash += tempFile.hashCode();
     return hash;
   }
@@ -318,12 +368,16 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
   /**
    * This method will read the records from sort temp file and keep it in a buffer
    *
-   * @param numberOfRecords number of records to be read
-   * @return batch of intermediate sort temp row
-   * @throws IOException if error occurs reading records from file
+   * @param numberOfRecords
+   * @return
+   * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow[] prefetchRecordsFromFile(int numberOfRecords)
-      throws IOException {
-    return readBatchedRowFromStream(numberOfRecords);
+  private Object[][] prefetchRecordsFromFile(int numberOfRecords)
+      throws CarbonSortKeyAndGroupByException {
+    Object[][] records = new Object[numberOfRecords][];
+    for (int i = 0; i < numberOfRecords; i++) {
+      records[i] = getRowFromStream();
+    }
+    return records;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
index 22673ff..4bbf61b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
@@ -21,21 +21,25 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
 import java.util.AbstractQueue;
+import java.util.Arrays;
 import java.util.PriorityQueue;
 import java.util.concurrent.Callable;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.SortTempChunkHolder;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeSortTempFileChunkHolder;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeIntermediateFileMerger implements Callable<Void> {
   /**
@@ -65,13 +69,22 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   private int totalNumberOfRecords;
 
   private SortParameters mergerParameters;
-  private TableFieldStat tableFieldStat;
+
   private File[] intermediateFiles;
+
   private File outPutFile;
 
+  private int dimCnt;
+  private int complexCnt;
+  private int measureCnt;
+  private boolean[] isNoDictionaryDimensionColumn;
+  private DataType[] measureDataTypes;
   private int writeBufferSize;
   private String compressorName;
-  private SortStepRowHandler sortStepRowHandler;
+
+  private long[] nullSetWords;
+
+  private ByteBuffer rowData;
 
   private Throwable throwable;
 
@@ -84,10 +97,16 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
     this.fileCounter = intermediateFiles.length;
     this.intermediateFiles = intermediateFiles;
     this.outPutFile = outPutFile;
+    this.dimCnt = mergerParameters.getDimColCount();
+    this.complexCnt = mergerParameters.getComplexDimColCount();
+    this.measureCnt = mergerParameters.getMeasureColCount();
+    this.isNoDictionaryDimensionColumn = mergerParameters.getNoDictionaryDimnesionColumn();
+    this.measureDataTypes = mergerParameters.getMeasureDataType();
     this.writeBufferSize = mergerParameters.getBufferSize();
     this.compressorName = mergerParameters.getSortTempCompressorName();
-    this.tableFieldStat = new TableFieldStat(mergerParameters);
-    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
+    this.nullSetWords = new long[((measureCnt - 1) >> 6) + 1];
+    // Take size of 2 MB for each row. I think it is high enough to use
+    rowData = ByteBuffer.allocate(2 * 1024 * 1024);
   }
 
   @Override public Void call() throws Exception {
@@ -146,14 +165,13 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get sorted sort temp row from the sort temp files
+   * This method will be used to get the sorted record from file
    *
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow getSortedRecordFromFile()
-      throws CarbonSortKeyAndGroupByException {
-    IntermediateSortTempRow row = null;
+  private Object[] getSortedRecordFromFile() throws CarbonSortKeyAndGroupByException {
+    Object[] row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will
@@ -217,7 +235,7 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
       this.recordHolderHeap.add(sortTempFileChunkHolder);
     }
 
-    LOGGER.info("Heap Size: " + this.recordHolderHeap.size());
+    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
   }
 
   /**
@@ -232,12 +250,12 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted sort temp row
+   * This method will be used to get the sorted row
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow next() throws CarbonSortKeyAndGroupByException {
+  private Object[] next() throws CarbonSortKeyAndGroupByException {
     return getSortedRecordFromFile();
   }
 
@@ -254,16 +272,82 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   /**
    * Below method will be used to write data to file
    *
-   * @throws IOException problem while writing
+   * @throws CarbonSortKeyAndGroupByException problem while writing
    */
-  private void writeDataToFile(IntermediateSortTempRow row) throws IOException {
-    sortStepRowHandler.writeIntermediateSortTempRowToOutputStream(row, stream);
+  private void writeDataToFile(Object[] row) throws CarbonSortKeyAndGroupByException, IOException {
+    int dimCount = 0;
+    int size = 0;
+    for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
+      if (isNoDictionaryDimensionColumn[dimCount]) {
+        byte[] col = (byte[]) row[dimCount];
+        rowData.putShort((short) col.length);
+        size += 2;
+        rowData.put(col);
+        size += col.length;
+      } else {
+        rowData.putInt((int) row[dimCount]);
+        size += 4;
+      }
+    }
+
+    // write complex dimensions here.
+    int dimensionSize = dimCnt + complexCnt;
+    for (; dimCount < dimensionSize; dimCount++) {
+      byte[] col = (byte[]) row[dimCount];
+      rowData.putShort((short)col.length);
+      size += 2;
+      rowData.put(col);
+      size += col.length;
+    }
+    Arrays.fill(nullSetWords, 0);
+    int nullSetSize = nullSetWords.length * 8;
+    int nullLoc = size;
+    size += nullSetSize;
+    for (int mesCount = 0; mesCount < measureCnt; mesCount++) {
+      Object value = row[mesCount + dimensionSize];
+      if (null != value) {
+        DataType dataType = measureDataTypes[mesCount];
+        if (dataType == DataTypes.SHORT) {
+          rowData.putShort(size, (Short) value);
+          size += 2;
+        } else if (dataType == DataTypes.INT) {
+          rowData.putInt(size, (Integer) value);
+          size += 4;
+        } else if (dataType == DataTypes.LONG) {
+          rowData.putLong(size, (Long) value);
+          size += 8;
+        } else if (dataType == DataTypes.DOUBLE) {
+          rowData.putDouble(size, (Double) value);
+          size += 8;
+        } else if (DataTypes.isDecimal(dataType)) {
+          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(((BigDecimal) value));
+          rowData.putShort(size, (short) bigDecimalInBytes.length);
+          size += 2;
+          for (int i = 0; i < bigDecimalInBytes.length; i++) {
+            rowData.put(size++, bigDecimalInBytes[i]);
+          }
+        }
+        UnsafeCarbonRowPage.set(nullSetWords, mesCount);
+      } else {
+        UnsafeCarbonRowPage.unset(nullSetWords, mesCount);
+      }
+    }
+    for (int i = 0; i < nullSetWords.length; i++) {
+      rowData.putLong(nullLoc, nullSetWords[i]);
+      nullLoc += 8;
+    }
+    byte[] rowBytes = new byte[size];
+    rowData.position(0);
+    rowData.get(rowBytes);
+    stream.write(rowBytes);
+    rowData.clear();
   }
 
   private void finish() throws CarbonSortKeyAndGroupByException {
     clear();
     try {
       CarbonUtil.deleteFiles(intermediateFiles);
+      rowData.clear();
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while deleting the intermediate files");
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
index 64f3c25..ce118d9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
@@ -29,8 +29,7 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.processing.loading.sort.SortStepRowUtil;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.SortTempChunkHolder;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeFinalMergePageHolder;
@@ -56,7 +55,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   private AbstractQueue<SortTempChunkHolder> recordHolderHeapLocal;
 
   private SortParameters parameters;
-  private SortStepRowHandler sortStepRowHandler;
+  private SortStepRowUtil sortStepRowUtil;
   /**
    * tempFileLocation
    */
@@ -69,7 +68,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   public UnsafeSingleThreadFinalSortFilesMerger(SortParameters parameters,
       String[] tempFileLocation) {
     this.parameters = parameters;
-    this.sortStepRowHandler = new SortStepRowHandler(parameters);
+    this.sortStepRowUtil = new SortStepRowUtil(parameters);
     this.tempFileLocation = tempFileLocation;
     this.tableName = parameters.getTableName();
   }
@@ -109,7 +108,9 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       LOGGER.info("Started adding first record from each page");
       for (final UnsafeCarbonRowPage rowPage : rowPages) {
 
-        SortTempChunkHolder sortTempFileChunkHolder = new UnsafeInmemoryHolder(rowPage);
+        SortTempChunkHolder sortTempFileChunkHolder = new UnsafeInmemoryHolder(rowPage,
+            parameters.getDimColCount() + parameters.getComplexDimColCount() + parameters
+                .getMeasureColCount(), parameters.getNumberOfSortColumns());
 
         // initialize
         sortTempFileChunkHolder.readRow();
@@ -120,7 +121,9 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       for (final UnsafeInMemoryIntermediateDataMerger merger : merges) {
 
         SortTempChunkHolder sortTempFileChunkHolder =
-            new UnsafeFinalMergePageHolder(merger, parameters.getNoDictionarySortColumn());
+            new UnsafeFinalMergePageHolder(merger, parameters.getNoDictionarySortColumn(),
+                parameters.getDimColCount() + parameters.getComplexDimColCount() + parameters
+                    .getMeasureColCount());
 
         // initialize
         sortTempFileChunkHolder.readRow();
@@ -139,7 +142,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
         recordHolderHeapLocal.add(sortTempFileChunkHolder);
       }
 
-      LOGGER.info("Heap Size: " + this.recordHolderHeapLocal.size());
+      LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
     } catch (Exception e) {
       LOGGER.error(e);
       throw new CarbonDataWriterException(e);
@@ -177,14 +180,12 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   }
 
   /**
-   * This method will be used to get the sorted row in 3-parted format.
-   * The row will feed the following writer process step.
+   * This method will be used to get the sorted row
    *
    * @return sorted row
    */
   public Object[] next() {
-    IntermediateSortTempRow sortTempRow =  getSortedRecordFromFile();
-    return sortStepRowHandler.convertIntermediateSortTempRowTo3Parted(sortTempRow);
+    return sortStepRowUtil.convertRow(getSortedRecordFromFile());
   }
 
   /**
@@ -192,8 +193,8 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
    *
    * @return sorted record sorted record
    */
-  private IntermediateSortTempRow getSortedRecordFromFile() throws CarbonDataWriterException {
-    IntermediateSortTempRow row = null;
+  private Object[] getSortedRecordFromFile() throws CarbonDataWriterException {
+    Object[] row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index ea11e22..751903a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -389,6 +389,7 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
           noDictionarySortColumnMapping, 0, noDictionarySortColumnMapping.length);
     }
     sortParameters.setNoDictionarySortColumn(noDictionarySortColumnMapping);
+
     String[] sortTempFileLocation = CarbonDataProcessorUtil.arrayAppend(tempStoreLocation,
         CarbonCommonConstants.FILE_SEPARATOR, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
     finalMerger =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
index c06819c..04efa1f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
@@ -21,6 +21,7 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.math.BigDecimal;
 import java.util.AbstractQueue;
 import java.util.PriorityQueue;
 import java.util.concurrent.Callable;
@@ -28,9 +29,11 @@ import java.util.concurrent.Callable;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.NonDictionaryUtil;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 public class IntermediateFileMerger implements Callable<Void> {
@@ -65,12 +68,17 @@ public class IntermediateFileMerger implements Callable<Void> {
   private File[] intermediateFiles;
 
   private File outPutFile;
+  private int dimCnt;
+  private int noDictDimCnt;
+  private int complexCnt;
+  private int measureCnt;
+  private boolean[] isNoDictionaryDimensionColumn;
+  private DataType[] measureDataTypes;
   private int writeBufferSize;
   private String compressorName;
 
   private Throwable throwable;
-  private TableFieldStat tableFieldStat;
-  private SortStepRowHandler sortStepRowHandler;
+
   /**
    * IntermediateFileMerger Constructor
    */
@@ -80,10 +88,14 @@ public class IntermediateFileMerger implements Callable<Void> {
     this.fileCounter = intermediateFiles.length;
     this.intermediateFiles = intermediateFiles;
     this.outPutFile = outPutFile;
+    this.dimCnt = mergerParameters.getDimColCount();
+    this.noDictDimCnt = mergerParameters.getNoDictionaryCount();
+    this.complexCnt = mergerParameters.getComplexDimColCount();
+    this.measureCnt = mergerParameters.getMeasureColCount();
+    this.isNoDictionaryDimensionColumn = mergerParameters.getNoDictionaryDimnesionColumn();
+    this.measureDataTypes = mergerParameters.getMeasureDataType();
     this.writeBufferSize = mergerParameters.getBufferSize();
     this.compressorName = mergerParameters.getSortTempCompressorName();
-    this.tableFieldStat = new TableFieldStat(mergerParameters);
-    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
   }
 
   @Override public Void call() throws Exception {
@@ -142,14 +154,13 @@ public class IntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted sort temp row from sort temp file
+   * This method will be used to get the sorted record from file
    *
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow getSortedRecordFromFile()
-      throws CarbonSortKeyAndGroupByException {
-    IntermediateSortTempRow row = null;
+  private Object[] getSortedRecordFromFile() throws CarbonSortKeyAndGroupByException {
+    Object[] row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will
@@ -216,7 +227,7 @@ public class IntermediateFileMerger implements Callable<Void> {
       this.recordHolderHeap.add(sortTempFileChunkHolder);
     }
 
-    LOGGER.info("Heap Size: " + this.recordHolderHeap.size());
+    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
   }
 
   /**
@@ -231,12 +242,12 @@ public class IntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted sort temp row
+   * This method will be used to get the sorted row
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow next() throws CarbonSortKeyAndGroupByException {
+  private Object[] next() throws CarbonSortKeyAndGroupByException {
     return getSortedRecordFromFile();
   }
 
@@ -253,10 +264,62 @@ public class IntermediateFileMerger implements Callable<Void> {
   /**
    * Below method will be used to write data to file
    *
-   * @throws IOException problem while writing
+   * @throws CarbonSortKeyAndGroupByException problem while writing
    */
-  private void writeDataToFile(IntermediateSortTempRow row) throws IOException {
-    sortStepRowHandler.writeIntermediateSortTempRowToOutputStream(row, stream);
+  private void writeDataToFile(Object[] row) throws CarbonSortKeyAndGroupByException {
+    try {
+      int[] mdkArray = (int[]) row[0];
+      byte[][] nonDictArray = (byte[][]) row[1];
+      int mdkIndex = 0;
+      int nonDictKeyIndex = 0;
+      // write dictionary and non dictionary dimensions here.
+      for (boolean nodictinary : isNoDictionaryDimensionColumn) {
+        if (nodictinary) {
+          byte[] col = nonDictArray[nonDictKeyIndex++];
+          stream.writeShort(col.length);
+          stream.write(col);
+        } else {
+          stream.writeInt(mdkArray[mdkIndex++]);
+        }
+      }
+      // write complex
+      for (; nonDictKeyIndex < noDictDimCnt + complexCnt; nonDictKeyIndex++) {
+        byte[] col = nonDictArray[nonDictKeyIndex++];
+        stream.writeShort(col.length);
+        stream.write(col);
+      }
+      // write measure
+      int fieldIndex = 0;
+      for (int counter = 0; counter < measureCnt; counter++) {
+        if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
+          stream.write((byte) 1);
+          DataType dataType = measureDataTypes[counter];
+          if (dataType == DataTypes.BOOLEAN) {
+            stream.writeBoolean((boolean)NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (dataType == DataTypes.SHORT) {
+            stream.writeShort((short) NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (dataType == DataTypes.INT) {
+            stream.writeInt((int) NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (dataType == DataTypes.LONG) {
+            stream.writeLong((long) NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (dataType == DataTypes.DOUBLE) {
+            stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
+          } else if (DataTypes.isDecimal(dataType)) {
+            byte[] bigDecimalInBytes = DataTypeUtil
+                .bigDecimalToByte((BigDecimal) NonDictionaryUtil.getMeasure(fieldIndex, row));
+            stream.writeInt(bigDecimalInBytes.length);
+            stream.write(bigDecimalInBytes);
+          } else {
+            throw new IllegalArgumentException("unsupported data type:" + dataType);
+          }
+        } else {
+          stream.write((byte) 0);
+        }
+        fieldIndex++;
+      }
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
+    }
   }
 
   private void finish() throws CarbonSortKeyAndGroupByException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
deleted file mode 100644
index 9b6d1e8..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sort.sortdata;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-
-/**
- * This class is used as comparator for comparing intermediate sort temp row
- */
-public class IntermediateSortTempRowComparator implements Comparator<IntermediateSortTempRow> {
-  /**
-   * isSortColumnNoDictionary whether the sort column is not dictionary or not
-   */
-  private boolean[] isSortColumnNoDictionary;
-
-  /**
-   * @param isSortColumnNoDictionary isSortColumnNoDictionary
-   */
-  public IntermediateSortTempRowComparator(boolean[] isSortColumnNoDictionary) {
-    this.isSortColumnNoDictionary = isSortColumnNoDictionary;
-  }
-
-  /**
-   * Below method will be used to compare two sort temp row
-   */
-  public int compare(IntermediateSortTempRow rowA, IntermediateSortTempRow rowB) {
-    int diff = 0;
-    int dictIndex = 0;
-    int nonDictIndex = 0;
-
-    for (boolean isNoDictionary : isSortColumnNoDictionary) {
-
-      if (isNoDictionary) {
-        byte[] byteArr1 = rowA.getNoDictSortDims()[nonDictIndex];
-        byte[] byteArr2 = rowB.getNoDictSortDims()[nonDictIndex];
-        nonDictIndex++;
-
-        int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
-        if (difference != 0) {
-          return difference;
-        }
-      } else {
-        int dimFieldA = rowA.getDictSortDims()[dictIndex];
-        int dimFieldB = rowB.getDictSortDims()[dictIndex];
-        dictIndex++;
-
-        diff = dimFieldA - dimFieldB;
-        if (diff != 0) {
-          return diff;
-        }
-      }
-    }
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
index 3f94533..d2579d2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
@@ -40,11 +40,14 @@ public class NewRowComparator implements Comparator<Object[]> {
    */
   public int compare(Object[] rowA, Object[] rowB) {
     int diff = 0;
+
     int index = 0;
 
     for (boolean isNoDictionary : noDictionarySortColumnMaping) {
+
       if (isNoDictionary) {
         byte[] byteArr1 = (byte[]) rowA[index];
+
         byte[] byteArr2 = (byte[]) rowB[index];
 
         int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
@@ -54,7 +57,6 @@ public class NewRowComparator implements Comparator<Object[]> {
       } else {
         int dimFieldA = (int) rowA[index];
         int dimFieldB = (int) rowB[index];
-
         diff = dimFieldA - dimFieldB;
         if (diff != 0) {
           return diff;
@@ -63,6 +65,7 @@ public class NewRowComparator implements Comparator<Object[]> {
 
       index++;
     }
+
     return diff;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
index 7538c92..e01b587 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
@@ -29,7 +29,7 @@ public class NewRowComparatorForNormalDims implements Comparator<Object[]> {
   private int numberOfSortColumns;
 
   /**
-   * NewRowComparatorForNormalDims Constructor
+   * RowComparatorForNormalDims Constructor
    *
    * @param numberOfSortColumns
    */
@@ -46,6 +46,7 @@ public class NewRowComparatorForNormalDims implements Comparator<Object[]> {
     int diff = 0;
 
     for (int i = 0; i < numberOfSortColumns; i++) {
+
       int dimFieldA = (int)rowA[i];
       int dimFieldB = (int)rowB[i];
       diff = dimFieldA - dimFieldB;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
new file mode 100644
index 0000000..0ae0b93
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.sort.sortdata;
+
+import java.nio.ByteBuffer;
+import java.util.Comparator;
+
+import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
+import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
+import org.apache.carbondata.core.util.NonDictionaryUtil;
+
+public class RowComparator implements Comparator<Object[]> {
+  /**
+   * noDictionaryCount represent number of no dictionary cols
+   */
+  private int noDictionaryCount;
+
+  /**
+   * noDictionaryColMaping mapping of dictionary dimensions and no dictionary dimensions.
+   */
+  private boolean[] noDictionarySortColumnMaping;
+
+  /**
+   * @param noDictionarySortColumnMaping
+   * @param noDictionaryCount
+   */
+  public RowComparator(boolean[] noDictionarySortColumnMaping, int noDictionaryCount) {
+    this.noDictionaryCount = noDictionaryCount;
+    this.noDictionarySortColumnMaping = noDictionarySortColumnMaping;
+  }
+
+  /**
+   * Below method will be used to compare two mdkey
+   */
+  public int compare(Object[] rowA, Object[] rowB) {
+    int diff = 0;
+
+    int normalIndex = 0;
+    int noDictionaryindex = 0;
+
+    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
+
+      if (isNoDictionary) {
+        byte[] byteArr1 = (byte[]) rowA[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+
+        ByteBuffer buff1 = ByteBuffer.wrap(byteArr1);
+
+        // extract a high card dims from complete byte[].
+        NonDictionaryUtil
+            .extractSingleHighCardDims(byteArr1, noDictionaryindex, noDictionaryCount, buff1);
+
+        byte[] byteArr2 = (byte[]) rowB[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+
+        ByteBuffer buff2 = ByteBuffer.wrap(byteArr2);
+
+        // extract a high card dims from complete byte[].
+        NonDictionaryUtil
+            .extractSingleHighCardDims(byteArr2, noDictionaryindex, noDictionaryCount, buff2);
+
+        int difference = UnsafeComparer.INSTANCE.compareTo(buff1, buff2);
+        if (difference != 0) {
+          return difference;
+        }
+        noDictionaryindex++;
+      } else {
+        int dimFieldA = NonDictionaryUtil.getDimension(normalIndex, rowA);
+        int dimFieldB = NonDictionaryUtil.getDimension(normalIndex, rowB);
+        diff = dimFieldA - dimFieldB;
+        if (diff != 0) {
+          return diff;
+        }
+        normalIndex++;
+      }
+
+    }
+
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
new file mode 100644
index 0000000..0883ae1
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.sort.sortdata;
+
+import java.util.Comparator;
+
+import org.apache.carbondata.core.util.NonDictionaryUtil;
+
+/**
+ * This class is used as comparator for comparing dims which are non high cardinality dims.
+ * Here the dims will be in form of int[] (surrogates) so directly comparing the integers.
+ */
+public class RowComparatorForNormalDims implements Comparator<Object[]> {
+  /**
+   * dimension count
+   */
+  private int numberOfSortColumns;
+
+  /**
+   * RowComparatorForNormalDims Constructor
+   *
+   * @param numberOfSortColumns
+   */
+  public RowComparatorForNormalDims(int numberOfSortColumns) {
+    this.numberOfSortColumns = numberOfSortColumns;
+  }
+
+  /**
+   * Below method will be used to compare two surrogate keys
+   *
+   * @see Comparator#compare(Object, Object)
+   */
+  public int compare(Object[] rowA, Object[] rowB) {
+    int diff = 0;
+
+    for (int i = 0; i < numberOfSortColumns; i++) {
+
+      int dimFieldA = NonDictionaryUtil.getDimension(i, rowA);
+      int dimFieldB = NonDictionaryUtil.getDimension(i, rowB);
+
+      diff = dimFieldA - dimFieldB;
+      if (diff != 0) {
+        return diff;
+      }
+    }
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
index a4ac0ea..88695b9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
@@ -37,8 +37,6 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
@@ -73,12 +71,12 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    * tableName
    */
   private String tableName;
-  private SortParameters sortParameters;
-  private SortStepRowHandler sortStepRowHandler;
+
   /**
    * tempFileLocation
    */
   private String[] tempFileLocation;
+  private SortParameters sortParameters;
 
   private int maxThreadForSorting;
 
@@ -91,7 +89,6 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
     this.tempFileLocation = tempFileLocation;
     this.tableName = tableName;
     this.sortParameters = sortParameters;
-    this.sortStepRowHandler = new SortStepRowHandler(sortParameters);
     try {
       maxThreadForSorting = Integer.parseInt(CarbonProperties.getInstance()
           .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD,
@@ -110,7 +107,8 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    */
   public void startFinalMerge() throws CarbonDataWriterException {
     List<File> filesToMerge = getFilesToMergeSort();
-    if (filesToMerge.size() == 0) {
+    if (filesToMerge.size() == 0)
+    {
       LOGGER.info("No file to merge in final merge stage");
       return;
     }
@@ -127,9 +125,11 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
 
     // get all the merged files
     List<File> files = new ArrayList<File>(tempFileLocation.length);
-    for (String tempLoc : tempFileLocation) {
+    for (String tempLoc : tempFileLocation)
+    {
       File[] subFiles = new File(tempLoc).listFiles(fileFilter);
-      if (null != subFiles && subFiles.length > 0) {
+      if (null != subFiles && subFiles.length > 0)
+      {
         files.addAll(Arrays.asList(subFiles));
       }
     }
@@ -226,14 +226,13 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
   }
 
   /**
-   * This method will be used to get the sorted sort temp row from the sort temp files
+   * This method will be used to get the sorted row
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
   public Object[] next() {
-    IntermediateSortTempRow sortTempRow = getSortedRecordFromFile();
-    return sortStepRowHandler.convertIntermediateSortTempRowTo3Parted(sortTempRow);
+    return getSortedRecordFromFile();
   }
 
   /**
@@ -242,8 +241,8 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow getSortedRecordFromFile() throws CarbonDataWriterException {
-    IntermediateSortTempRow row = null;
+  private Object[] getSortedRecordFromFile() throws CarbonDataWriterException {
+    Object[] row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
index c7efbd9..57a19bd 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.processing.sort.sortdata;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
-import java.nio.ByteBuffer;
+import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
@@ -32,10 +32,12 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
@@ -67,8 +69,7 @@ public class SortDataRows {
   private Semaphore semaphore;
 
   private SortParameters parameters;
-  private SortStepRowHandler sortStepRowHandler;
-  private ThreadLocal<ByteBuffer> rowBuffer;
+
   private int sortBufferSize;
 
   private SortIntermediateFileMerger intermediateFileMerger;
@@ -78,7 +79,7 @@ public class SortDataRows {
   public SortDataRows(SortParameters parameters,
       SortIntermediateFileMerger intermediateFileMerger) {
     this.parameters = parameters;
-    this.sortStepRowHandler = new SortStepRowHandler(parameters);
+
     this.intermediateFileMerger = intermediateFileMerger;
 
     int batchSize = CarbonProperties.getInstance().getBatchSize();
@@ -86,12 +87,6 @@ public class SortDataRows {
     this.sortBufferSize = Math.max(parameters.getSortBufferSize(), batchSize);
     // observer of writing file in thread
     this.threadStatusObserver = new ThreadStatusObserver();
-    this.rowBuffer = new ThreadLocal<ByteBuffer>() {
-      @Override protected ByteBuffer initialValue() {
-        byte[] backedArray = new byte[2 * 1024 * 1024];
-        return ByteBuffer.wrap(backedArray);
-      }
-    };
   }
 
   /**
@@ -135,7 +130,8 @@ public class SortDataRows {
         semaphore.acquire();
         dataSorterAndWriterExecutorService.execute(new DataSorterAndWriter(recordHolderListLocal));
       } catch (InterruptedException e) {
-        LOGGER.error(e, "exception occurred while trying to acquire a semaphore lock: ");
+        LOGGER.error(e,
+            "exception occurred while trying to acquire a semaphore lock: ");
         throw new CarbonSortKeyAndGroupByException(e);
       }
       // create the new holder Array
@@ -162,7 +158,7 @@ public class SortDataRows {
         }
         intermediateFileMerger.startMergingIfPossible();
         Object[][] recordHolderListLocal = recordHolderList;
-        sizeLeft = sortBufferSize - entryCount;
+        sizeLeft = sortBufferSize - entryCount ;
         if (sizeLeft > 0) {
           System.arraycopy(rowBatch, 0, recordHolderListLocal, entryCount, sizeLeft);
         }
@@ -216,6 +212,7 @@ public class SortDataRows {
           locationChosen + File.separator + parameters.getTableName() +
               System.nanoTime() + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
       writeDataToFile(recordHolderList, this.entryCount, file);
+
     }
 
     startFileBasedMerge();
@@ -223,7 +220,7 @@ public class SortDataRows {
   }
 
   /**
-   * Below method will be used to write data to sort temp file
+   * Below method will be used to write data to file
    *
    * @throws CarbonSortKeyAndGroupByException problem while writing
    */
@@ -236,9 +233,60 @@ public class SortDataRows {
           parameters.getFileWriteBufferSize(), parameters.getSortTempCompressorName());
       // write number of entries to the file
       stream.writeInt(entryCountLocal);
+      int complexDimColCount = parameters.getComplexDimColCount();
+      int dimColCount = parameters.getDimColCount() + complexDimColCount;
+      DataType[] type = parameters.getMeasureDataType();
+      boolean[] noDictionaryDimnesionMapping = parameters.getNoDictionaryDimnesionColumn();
+      Object[] row = null;
       for (int i = 0; i < entryCountLocal; i++) {
-        sortStepRowHandler.writeRawRowAsIntermediateSortTempRowToOutputStream(
-            recordHolderList[i], stream, rowBuffer.get());
+        // get row from record holder list
+        row = recordHolderList[i];
+        int dimCount = 0;
+        // write dictionary and non dictionary dimensions here.
+        for (; dimCount < noDictionaryDimnesionMapping.length; dimCount++) {
+          if (noDictionaryDimnesionMapping[dimCount]) {
+            byte[] col = (byte[]) row[dimCount];
+            stream.writeShort(col.length);
+            stream.write(col);
+          } else {
+            stream.writeInt((int)row[dimCount]);
+          }
+        }
+        // write complex dimensions here.
+        for (; dimCount < dimColCount; dimCount++) {
+          byte[] value = (byte[])row[dimCount];
+          stream.writeShort(value.length);
+          stream.write(value);
+        }
+        // as measures are stored in separate array.
+        for (int mesCount = 0;
+             mesCount < parameters.getMeasureColCount(); mesCount++) {
+          Object value = row[mesCount + dimColCount];
+          if (null != value) {
+            stream.write((byte) 1);
+            DataType dataType = type[mesCount];
+            if (dataType == DataTypes.BOOLEAN) {
+              stream.writeBoolean((boolean) value);
+            } else if (dataType == DataTypes.SHORT) {
+              stream.writeShort((Short) value);
+            } else if (dataType == DataTypes.INT) {
+              stream.writeInt((Integer) value);
+            } else if (dataType == DataTypes.LONG) {
+              stream.writeLong((Long) value);
+            } else if (dataType == DataTypes.DOUBLE) {
+              stream.writeDouble((Double) value);
+            } else if (DataTypes.isDecimal(dataType)) {
+              BigDecimal val = (BigDecimal) value;
+              byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
+              stream.writeInt(bigDecimalInBytes.length);
+              stream.write(bigDecimalInBytes);
+            } else {
+              throw new IllegalArgumentException("unsupported data type:" + type[mesCount]);
+            }
+          } else {
+            stream.write((byte) 0);
+          }
+        }
       }
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
@@ -253,7 +301,7 @@ public class SortDataRows {
    *
    * @throws CarbonSortKeyAndGroupByException
    */
-  private void deleteSortLocationIfExists() throws CarbonSortKeyAndGroupByException {
+  public void deleteSortLocationIfExists() throws CarbonSortKeyAndGroupByException {
     CarbonDataProcessorUtil.deleteSortLocationIfExists(parameters.getTempFileLocation());
   }
 
@@ -332,8 +380,7 @@ public class SortDataRows {
         // intermediate merging of sort temp files will be triggered
         intermediateFileMerger.addFileToMerge(sortTempFile);
         LOGGER.info("Time taken to sort and write sort temp file " + sortTempFile + " is: " + (
-            System.currentTimeMillis() - startTime) + ", sort temp file size in MB is "
-            + sortTempFile.length() * 0.1 * 10 / 1024 / 1024);
+            System.currentTimeMillis() - startTime));
       } catch (Throwable e) {
         try {
           threadStatusObserver.notifyFailed(e);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
index 7e221a7..d726539 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
@@ -21,7 +21,6 @@ import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.Comparator;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -31,11 +30,14 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
-import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.NonDictionaryUtil;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHolder> {
@@ -69,13 +71,20 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   /**
    * return row
    */
-  private IntermediateSortTempRow returnRow;
+  private Object[] returnRow;
+  private int dimCnt;
+  private int noDictDimCnt;
+  private int complexCnt;
+  private int measureCnt;
+  private boolean[] isNoDictionaryDimensionColumn;
+  private boolean[] isNoDictionarySortColumn;
+  private DataType[] measureDataTypes;
   private int readBufferSize;
   private String compressorName;
 
-  private IntermediateSortTempRow[] currentBuffer;
+  private Object[][] currentBuffer;
 
-  private IntermediateSortTempRow[] backupBuffer;
+  private Object[][] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -95,9 +104,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
    * totalRecordFetch
    */
   private int totalRecordFetch;
-  private TableFieldStat tableFieldStat;
-  private SortStepRowHandler sortStepRowHandler;
-  private Comparator<IntermediateSortTempRow> comparator;
+
   /**
    * Constructor to initialize
    *
@@ -108,12 +115,16 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   public SortTempFileChunkHolder(File tempFile, SortParameters sortParameters, String tableName) {
     // set temp file
     this.tempFile = tempFile;
+    this.dimCnt = sortParameters.getDimColCount();
+    this.noDictDimCnt = sortParameters.getNoDictionaryCount();
+    this.complexCnt = sortParameters.getComplexDimColCount();
+    this.measureCnt = sortParameters.getMeasureColCount();
+    this.isNoDictionaryDimensionColumn = sortParameters.getNoDictionaryDimnesionColumn();
+    this.isNoDictionarySortColumn = sortParameters.getNoDictionarySortColumn();
+    this.measureDataTypes = sortParameters.getMeasureDataType();
     this.readBufferSize = sortParameters.getBufferSize();
     this.compressorName = sortParameters.getSortTempCompressorName();
-    this.tableFieldStat = new TableFieldStat(sortParameters);
-    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
-    this.comparator = new IntermediateSortTempRowComparator(
-        tableFieldStat.getIsSortColNoDictFlags());
+
     this.executorService = Executors
         .newFixedThreadPool(1, new CarbonThreadFactory("SafeSortTempChunkHolderPool:" + tableName));
   }
@@ -167,12 +178,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
     if (prefetch) {
       fillDataForPrefetch();
     } else {
-      try {
-        this.returnRow = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
-        this.numberOfObjectRead++;
-      } catch (IOException e) {
-        throw new CarbonSortKeyAndGroupByException("Problem while reading rows", e);
-      }
+      this.returnRow = getRowFromStream();
     }
   }
 
@@ -206,28 +212,86 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   }
 
   /**
-   * Read a batch of row from stream
-   *
+   * Reads row from file
    * @return Object[]
-   * @throws IOException if error occurs while reading from stream
+   * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow[] readBatchedRowFromStream(int expected) throws IOException {
-    IntermediateSortTempRow[] holders = new IntermediateSortTempRow[expected];
-    for (int i = 0; i < expected; i++) {
-      IntermediateSortTempRow holder
-          = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
-      holders[i] = holder;
+  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
+    // create new row of size 3 (1 for dims , 1 for high card , 1 for measures)
+
+    Object[] holder = new Object[3];
+    int index = 0;
+    int nonDicIndex = 0;
+    int[] dim = new int[dimCnt - noDictDimCnt];
+    byte[][] nonDicArray = new byte[noDictDimCnt + complexCnt][];
+    Object[] measures = new Object[measureCnt];
+    try {
+      // read dimension values
+      for (int i = 0; i < isNoDictionaryDimensionColumn.length; i++) {
+        if (isNoDictionaryDimensionColumn[i]) {
+          short len = stream.readShort();
+          byte[] array = new byte[len];
+          stream.readFully(array);
+          nonDicArray[nonDicIndex++] = array;
+        } else {
+          dim[index++] = stream.readInt();
+        }
+      }
+
+      for (int i = 0; i < complexCnt; i++) {
+        short len = stream.readShort();
+        byte[] array = new byte[len];
+        stream.readFully(array);
+        nonDicArray[nonDicIndex++] = array;
+      }
+
+      index = 0;
+      // read measure values
+      for (int i = 0; i < measureCnt; i++) {
+        if (stream.readByte() == 1) {
+          DataType dataType = measureDataTypes[i];
+          if (dataType == DataTypes.BOOLEAN) {
+            measures[index++] = stream.readBoolean();
+          } else if (dataType == DataTypes.SHORT) {
+            measures[index++] = stream.readShort();
+          } else if (dataType == DataTypes.INT) {
+            measures[index++] = stream.readInt();
+          } else if (dataType == DataTypes.LONG) {
+            measures[index++] = stream.readLong();
+          } else if (dataType == DataTypes.DOUBLE) {
+            measures[index++] = stream.readDouble();
+          } else if (DataTypes.isDecimal(dataType)) {
+            int len = stream.readInt();
+            byte[] buff = new byte[len];
+            stream.readFully(buff);
+            measures[index++] = DataTypeUtil.byteToBigDecimal(buff);
+          } else {
+            throw new IllegalArgumentException("unsupported data type:" + dataType);
+          }
+        } else {
+          measures[index++] = null;
+        }
+      }
+
+      NonDictionaryUtil.prepareOutObj(holder, dim, nonDicArray, measures);
+
+      // increment number if record read
+      this.numberOfObjectRead++;
+    } catch (IOException e) {
+      LOGGER.error("Problme while reading the madkey fom sort temp file");
+      throw new CarbonSortKeyAndGroupByException("Problem while reading the sort temp file ", e);
     }
-    this.numberOfObjectRead += expected;
-    return holders;
+
+    //return out row
+    return holder;
   }
 
   /**
-   * below method will be used to get the sort temp row
+   * below method will be used to get the row
    *
    * @return row
    */
-  public IntermediateSortTempRow getRow() {
+  public Object[] getRow() {
     return this.returnRow;
   }
 
@@ -266,7 +330,31 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   }
 
   @Override public int compareTo(SortTempFileChunkHolder other) {
-    return comparator.compare(returnRow, other.getRow());
+    int diff = 0;
+    int index = 0;
+    int noDictionaryIndex = 0;
+    int[] leftMdkArray = (int[]) returnRow[0];
+    int[] rightMdkArray = (int[]) other.returnRow[0];
+    byte[][] leftNonDictArray = (byte[][]) returnRow[1];
+    byte[][] rightNonDictArray = (byte[][]) other.returnRow[1];
+    for (boolean isNoDictionary : isNoDictionarySortColumn) {
+      if (isNoDictionary) {
+        diff = UnsafeComparer.INSTANCE
+            .compareTo(leftNonDictArray[noDictionaryIndex], rightNonDictArray[noDictionaryIndex]);
+        if (diff != 0) {
+          return diff;
+        }
+        noDictionaryIndex++;
+      } else {
+        diff = leftMdkArray[index] - rightMdkArray[index];
+        if (diff != 0) {
+          return diff;
+        }
+        index++;
+      }
+
+    }
+    return diff;
   }
 
   @Override public boolean equals(Object obj) {
@@ -284,7 +372,9 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
 
   @Override public int hashCode() {
     int hash = 0;
-    hash += tableFieldStat.hashCode();
+    hash += 31 * measureCnt;
+    hash += 31 * dimCnt;
+    hash += 31 * complexCnt;
     hash += tempFile.hashCode();
     return hash;
   }
@@ -324,12 +414,16 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   /**
    * This method will read the records from sort temp file and keep it in a buffer
    *
-   * @param numberOfRecords number of records to be read
-   * @return batch of intermediate sort temp row
-   * @throws IOException if error occurs while reading reading records
+   * @param numberOfRecords
+   * @return
+   * @throws CarbonSortKeyAndGroupByException
    */
-  private IntermediateSortTempRow[] prefetchRecordsFromFile(int numberOfRecords)
-      throws IOException {
-    return readBatchedRowFromStream(numberOfRecords);
+  private Object[][] prefetchRecordsFromFile(int numberOfRecords)
+      throws CarbonSortKeyAndGroupByException {
+    Object[][] records = new Object[numberOfRecords][];
+    for (int i = 0; i < numberOfRecords; i++) {
+      records[i] = getRowFromStream();
+    }
+    return records;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2058a472/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
deleted file mode 100644
index 0d1303a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sort.sortdata;
-
-import java.io.Serializable;
-import java.util.Objects;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-
-/**
- * This class is used to hold field information for a table during data loading. These information
- * will be used to convert/construct/destruct row in sort process step. Because complex field is
- * processed the same as no-dict-no-sort-simple-dimension, so we treat them as the same and use
- * `no-dict-no-sort-dim` related variable to represent them in this class.
- */
-public class TableFieldStat implements Serializable {
-  private static final long serialVersionUID = 201712070950L;
-  private int dictSortDimCnt = 0;
-  private int dictNoSortDimCnt = 0;
-  private int noDictSortDimCnt = 0;
-  private int noDictNoSortDimCnt = 0;
-  // whether sort column is of dictionary type or not
-  private boolean[] isSortColNoDictFlags;
-  private int measureCnt;
-  private DataType[] measureDataType;
-
-  // indices for dict & sort dimension columns
-  private int[] dictSortDimIdx;
-  // indices for dict & no-sort dimension columns
-  private int[] dictNoSortDimIdx;
-  // indices for no-dict & sort dimension columns
-  private int[] noDictSortDimIdx;
-  // indices for no-dict & no-sort dimension columns, including complex columns
-  private int[] noDictNoSortDimIdx;
-  // indices for measure columns
-  private int[] measureIdx;
-
-  public TableFieldStat(SortParameters sortParameters) {
-    int noDictDimCnt = sortParameters.getNoDictionaryCount();
-    int complexDimCnt = sortParameters.getComplexDimColCount();
-    int dictDimCnt = sortParameters.getDimColCount() - noDictDimCnt;
-    this.isSortColNoDictFlags = sortParameters.getNoDictionarySortColumn();
-    int sortColCnt = isSortColNoDictFlags.length;
-    for (boolean flag : isSortColNoDictFlags) {
-      if (flag) {
-        noDictSortDimCnt++;
-      } else {
-        dictSortDimCnt++;
-      }
-    }
-    this.measureCnt = sortParameters.getMeasureColCount();
-    this.measureDataType = sortParameters.getMeasureDataType();
-
-    // be careful that the default value is 0
-    this.dictSortDimIdx = new int[dictSortDimCnt];
-    this.dictNoSortDimIdx = new int[dictDimCnt - dictSortDimCnt];
-    this.noDictSortDimIdx = new int[noDictSortDimCnt];
-    this.noDictNoSortDimIdx = new int[noDictDimCnt + complexDimCnt - noDictSortDimCnt];
-    this.measureIdx = new int[measureCnt];
-
-    int tmpNoDictSortCnt = 0;
-    int tmpNoDictNoSortCnt = 0;
-    int tmpDictSortCnt = 0;
-    int tmpDictNoSortCnt = 0;
-    boolean[] isDimNoDictFlags = sortParameters.getNoDictionaryDimnesionColumn();
-
-    for (int i = 0; i < isDimNoDictFlags.length; i++) {
-      if (isDimNoDictFlags[i]) {
-        if (i < sortColCnt && isSortColNoDictFlags[i]) {
-          noDictSortDimIdx[tmpNoDictSortCnt++] = i;
-        } else {
-          noDictNoSortDimIdx[tmpNoDictNoSortCnt++] = i;
-        }
-      } else {
-        if (i < sortColCnt && !isSortColNoDictFlags[i]) {
-          dictSortDimIdx[tmpDictSortCnt++] = i;
-        } else {
-          dictNoSortDimIdx[tmpDictNoSortCnt++] = i;
-        }
-      }
-    }
-    dictNoSortDimCnt = tmpDictNoSortCnt;
-
-    int base = isDimNoDictFlags.length;
-    // adding complex dimension columns
-    for (int i = 0; i < complexDimCnt; i++) {
-      noDictNoSortDimIdx[tmpNoDictNoSortCnt++] = base + i;
-    }
-    noDictNoSortDimCnt = tmpNoDictNoSortCnt;
-
-    base += complexDimCnt;
-    // indices for measure columns
-    for (int i = 0; i < measureCnt; i++) {
-      measureIdx[i] = base + i;
-    }
-  }
-
-  public int getDictSortDimCnt() {
-    return dictSortDimCnt;
-  }
-
-  public int getDictNoSortDimCnt() {
-    return dictNoSortDimCnt;
-  }
-
-  public int getNoDictSortDimCnt() {
-    return noDictSortDimCnt;
-  }
-
-  public int getNoDictNoSortDimCnt() {
-    return noDictNoSortDimCnt;
-  }
-
-  public boolean[] getIsSortColNoDictFlags() {
-    return isSortColNoDictFlags;
-  }
-
-  public int getMeasureCnt() {
-    return measureCnt;
-  }
-
-  public DataType[] getMeasureDataType() {
-    return measureDataType;
-  }
-
-  public int[] getDictSortDimIdx() {
-    return dictSortDimIdx;
-  }
-
-  public int[] getDictNoSortDimIdx() {
-    return dictNoSortDimIdx;
-  }
-
-  public int[] getNoDictSortDimIdx() {
-    return noDictSortDimIdx;
-  }
-
-  public int[] getNoDictNoSortDimIdx() {
-    return noDictNoSortDimIdx;
-  }
-
-  public int[] getMeasureIdx() {
-    return measureIdx;
-  }
-
-  @Override public boolean equals(Object o) {
-    if (this == o) return true;
-    if (!(o instanceof TableFieldStat)) return false;
-    TableFieldStat that = (TableFieldStat) o;
-    return dictSortDimCnt == that.dictSortDimCnt
-        && dictNoSortDimCnt == that.dictNoSortDimCnt
-        && noDictSortDimCnt == that.noDictSortDimCnt
-        && noDictNoSortDimCnt == that.noDictNoSortDimCnt
-        && measureCnt == that.measureCnt;
-  }
-
-  @Override public int hashCode() {
-    return Objects.hash(dictSortDimCnt, dictNoSortDimCnt, noDictSortDimCnt,
-        noDictNoSortDimCnt, measureCnt);
-  }
-}
\ No newline at end of file


[09/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
index 553f85e..773fbd7 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/PartitionSpliterRawResultIterator.java
@@ -17,20 +17,15 @@
 package org.apache.carbondata.core.scan.result.iterator;
 
 import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 
 public class PartitionSpliterRawResultIterator extends CarbonIterator<Object[]> {
 
-  private CarbonIterator<BatchResult> iterator;
-  private BatchResult batch;
+  private CarbonIterator<RowBatch> iterator;
+  private RowBatch batch;
   private int counter;
 
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(PartitionSpliterRawResultIterator.class.getName());
-
-  public PartitionSpliterRawResultIterator(CarbonIterator<BatchResult> iterator) {
+  public PartitionSpliterRawResultIterator(CarbonIterator<RowBatch> iterator) {
     this.iterator = iterator;
   }
 
@@ -65,7 +60,7 @@ public class PartitionSpliterRawResultIterator extends CarbonIterator<Object[]>
    * @param batch
    * @return
    */
-  private boolean checkBatchEnd(BatchResult batch) {
+  private boolean checkBatchEnd(RowBatch batch) {
     return !(counter < batch.getSize());
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
index 70d0958..1dd1595 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/RawResultIterator.java
@@ -21,7 +21,7 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 
 /**
@@ -37,7 +37,7 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
   /**
    * Iterator of the Batch raw result.
    */
-  private CarbonIterator<BatchResult> detailRawQueryResultIterator;
+  private CarbonIterator<RowBatch> detailRawQueryResultIterator;
 
   /**
    * Counter to maintain the row counter.
@@ -55,9 +55,9 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
   /**
    * batch of the result.
    */
-  private BatchResult batch;
+  private RowBatch batch;
 
-  public RawResultIterator(CarbonIterator<BatchResult> detailRawQueryResultIterator,
+  public RawResultIterator(CarbonIterator<RowBatch> detailRawQueryResultIterator,
       SegmentProperties sourceSegProperties, SegmentProperties destinationSegProperties) {
     this.detailRawQueryResultIterator = detailRawQueryResultIterator;
     this.sourceSegProperties = sourceSegProperties;
@@ -155,7 +155,7 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
    * @param batch
    * @return
    */
-  private boolean checkIfBatchIsProcessedCompletely(BatchResult batch) {
+  private boolean checkIfBatchIsProcessedCompletely(RowBatch batch) {
     if (counter < batch.getSize()) {
       return false;
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
index cc9710e..c7cb00d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/VectorDetailQueryResultIterator.java
@@ -35,10 +35,12 @@ public class VectorDetailQueryResultIterator extends AbstractDetailQueryResultIt
     super(infos, queryModel, execService);
   }
 
-  @Override public Object next() {
+  @Override
+  public Object next() {
     throw new UnsupportedOperationException("call processNextBatch instead");
   }
 
+  @Override
   public void processNextBatch(CarbonColumnarBatch columnarBatch) {
     synchronized (lock) {
       updateDataBlockIterator();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java
index cfc2f16..973ce0f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/CarbonColumnarBatch.java
@@ -87,7 +87,4 @@ public class CarbonColumnarBatch {
     }
   }
 
-  public int getRowsFilteredCount() {
-    return rowsFiltered;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
index a5f81b9..59117dd 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/ColumnVectorInfo.java
@@ -18,16 +18,16 @@ package org.apache.carbondata.core.scan.result.vector;
 
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 
 public class ColumnVectorInfo implements Comparable<ColumnVectorInfo> {
   public int offset;
   public int size;
   public CarbonColumnVector vector;
   public int vectorOffset;
-  public QueryDimension dimension;
-  public QueryMeasure measure;
+  public ProjectionDimension dimension;
+  public ProjectionMeasure measure;
   public int ordinal;
   public DirectDictionaryGenerator directDictionaryGenerator;
   public MeasureDataVectorProcessor.MeasureVectorFiller measureVectorFiller;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
index db4c982..8902dfb 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/vector/MeasureDataVectorProcessor.java
@@ -29,7 +29,7 @@ public class MeasureDataVectorProcessor {
 
     void fillMeasureVector(ColumnPage dataChunk, ColumnVectorInfo info);
 
-    void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info);
   }
 
@@ -60,7 +60,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -69,13 +69,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putInt(vectorOffset, (int)dataChunk.getLong(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {
@@ -117,7 +117,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping,
+    public void fillMeasureVector(int[] filteredRowId,
         ColumnPage dataChunk, ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -126,13 +126,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putBoolean(vectorOffset, dataChunk.getBoolean(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {
@@ -171,7 +171,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -180,13 +180,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putShort(vectorOffset, (short) dataChunk.getLong(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {
@@ -225,7 +225,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -234,13 +234,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putLong(vectorOffset, dataChunk.getLong(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {
@@ -279,7 +279,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -288,7 +288,7 @@ public class MeasureDataVectorProcessor {
       int precision = info.measure.getMeasure().getPrecision();
       BitSet nullBitSet = dataChunk.getNullBits();
       for (int i = offset; i < len; i++) {
-        int currentRow = rowMapping[i];
+        int currentRow = filteredRowId[i];
         if (nullBitSet.get(currentRow)) {
           vector.putNull(vectorOffset);
         } else {
@@ -330,7 +330,7 @@ public class MeasureDataVectorProcessor {
     }
 
     @Override
-    public void fillMeasureVectorForFilter(int[] rowMapping, ColumnPage dataChunk,
+    public void fillMeasureVector(int[] filteredRowId, ColumnPage dataChunk,
         ColumnVectorInfo info) {
       int offset = info.offset;
       int len = offset + info.size;
@@ -339,13 +339,13 @@ public class MeasureDataVectorProcessor {
       BitSet nullBitSet = dataChunk.getNullBits();
       if (nullBitSet.isEmpty()) {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           vector.putDouble(vectorOffset, dataChunk.getDouble(currentRow));
           vectorOffset++;
         }
       } else {
         for (int i = offset; i < len; i++) {
-          int currentRow = rowMapping[i];
+          int currentRow = filteredRowId[i];
           if (nullBitSet.get(currentRow)) {
             vector.putNull(vectorOffset);
           } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java
deleted file mode 100644
index bf26ca3..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/AbstractBlockletScanner.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.scanner;
-
-import java.io.IOException;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.datastore.page.ColumnPage;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
-import org.apache.carbondata.core.scan.result.impl.NonFilterQueryScannedResult;
-import org.apache.carbondata.core.stats.QueryStatistic;
-import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-
-/**
- * Blocklet scanner class to process the block
- */
-public abstract class AbstractBlockletScanner implements BlockletScanner {
-
-  /**
-   * block execution info
-   */
-  protected BlockExecutionInfo blockExecutionInfo;
-
-  public QueryStatisticsModel queryStatisticsModel;
-
-  private AbstractScannedResult emptyResult;
-
-  public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) {
-    this.blockExecutionInfo = tableBlockExecutionInfos;
-  }
-
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws IOException, FilterUnsupportedException {
-    long startTime = System.currentTimeMillis();
-    AbstractScannedResult scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
-    QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
-    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
-        totalBlockletStatistic.getCount() + 1);
-    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
-    validScannedBlockletStatistic
-        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
-            validScannedBlockletStatistic.getCount() + 1);
-    // adding statistics for valid number of pages
-    QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
-    validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
-        validPages.getCount() + blocksChunkHolder.getDataBlock().numberOfPages());
-    // adding statistics for number of pages
-    QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
-    totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
-        totalPagesScanned.getCount() + blocksChunkHolder.getDataBlock().numberOfPages());
-    scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR + blocksChunkHolder
-            .getDataBlock().blockletId());
-    if (!blockExecutionInfo.isPrefetchBlocklet()) {
-      readBlocklet(blocksChunkHolder);
-    }
-    DimensionRawColumnChunk[] dimensionRawColumnChunks =
-        blocksChunkHolder.getDimensionRawDataChunk();
-    DimensionColumnDataChunk[][] dimensionColumnDataChunks =
-        new DimensionColumnDataChunk[dimensionRawColumnChunks.length][blocksChunkHolder
-            .getDataBlock().numberOfPages()];
-    MeasureRawColumnChunk[] measureRawColumnChunks = blocksChunkHolder.getMeasureRawDataChunk();
-    ColumnPage[][] columnPages =
-        new ColumnPage[measureRawColumnChunks.length][blocksChunkHolder.getDataBlock()
-                       .numberOfPages()];
-    scannedResult.setDimensionChunks(dimensionColumnDataChunks);
-    scannedResult.setMeasureChunks(columnPages);
-    scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks);
-    scannedResult.setMsrRawColumnChunks(measureRawColumnChunks);
-    if (blockExecutionInfo.isPrefetchBlocklet()) {
-      for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
-        if (dimensionRawColumnChunks[i] != null) {
-          dimensionColumnDataChunks[i] = dimensionRawColumnChunks[i].convertToDimColDataChunks();
-        }
-      }
-      for (int i = 0; i < measureRawColumnChunks.length; i++) {
-        if (measureRawColumnChunks[i] != null) {
-          columnPages[i] = measureRawColumnChunks[i].convertToColumnPage();
-        }
-      }
-    }
-    int[] numberOfRows = null;
-    if (blockExecutionInfo.getAllSelectedDimensionBlocksIndexes().length > 0) {
-      for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
-        if (dimensionRawColumnChunks[i] != null) {
-          numberOfRows = dimensionRawColumnChunks[i].getRowCount();
-          break;
-        }
-      }
-    } else if (blockExecutionInfo.getAllSelectedMeasureBlocksIndexes().length > 0) {
-      for (int i = 0; i < measureRawColumnChunks.length; i++) {
-        if (measureRawColumnChunks[i] != null) {
-          numberOfRows = measureRawColumnChunks[i].getRowCount();
-          break;
-        }
-      }
-    }
-
-    // count(*)  case there would not be any dimensions are measures selected.
-    if (numberOfRows == null) {
-      numberOfRows = new int[blocksChunkHolder.getDataBlock().numberOfPages()];
-      for (int i = 0; i < numberOfRows.length; i++) {
-        numberOfRows[i] =
-            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      }
-      int lastPageSize = blocksChunkHolder.getDataBlock().nodeSize()
-          % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
-      ;
-      if (lastPageSize > 0) {
-        numberOfRows[numberOfRows.length - 1] = lastPageSize;
-      }
-    }
-    scannedResult.setNumberOfRows(numberOfRows);
-    if (!blockExecutionInfo.isPrefetchBlocklet()) {
-      scannedResult.fillDataChunks();
-    }
-    // adding statistics for carbon scan time
-    QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
-    scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
-        scanTime.getCount() + (System.currentTimeMillis() - startTime));
-    return scannedResult;
-  }
-
-  @Override public void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    long startTime = System.currentTimeMillis();
-    DimensionRawColumnChunk[] dimensionRawColumnChunks = blocksChunkHolder.getDataBlock()
-        .getDimensionChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedDimensionBlocksIndexes());
-    blocksChunkHolder.setDimensionRawDataChunk(dimensionRawColumnChunks);
-    MeasureRawColumnChunk[] measureRawColumnChunks = blocksChunkHolder.getDataBlock()
-        .getMeasureChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedMeasureBlocksIndexes());
-    blocksChunkHolder.setMeasureRawDataChunk(measureRawColumnChunks);
-    // adding statistics for carbon read time
-    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
-    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
-        readTime.getCount() + (System.currentTimeMillis() - startTime));
-  }
-
-  @Override public AbstractScannedResult createEmptyResult() {
-    if (emptyResult == null) {
-      emptyResult = new NonFilterQueryScannedResult(blockExecutionInfo);
-      emptyResult.setNumberOfRows(new int[0]);
-      emptyResult.setIndexes(new int[0][]);
-    }
-    return emptyResult;
-  }
-
-  @Override public boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    // For non filter it is always true
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java
index 0ed0d43..0a41032 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/BlockletScanner.java
@@ -18,9 +18,10 @@ package org.apache.carbondata.core.scan.scanner;
 
 import java.io.IOException;
 
+import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 
 /**
  * Interface for processing the block
@@ -30,31 +31,26 @@ public interface BlockletScanner {
 
   /**
    * Checks whether this blocklet required to scan or not based on min max of each blocklet.
-   * @param blocksChunkHolder
+   * @param dataBlock
    * @return
    * @throws IOException
    */
-  boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException;
+  boolean isScanRequired(DataRefNode dataBlock);
 
   /**
    * Below method will used to process the block data and get the scanned result
    *
-   * @param blocksChunkHolder block chunk which holds the block data
+   * @param rawBlockletColumnChunks block chunk which holds the block data
    * @return scannerResult
    * result after processing
    */
-  AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
+  BlockletScannedResult scanBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks)
       throws IOException, FilterUnsupportedException;
 
   /**
    * Just reads the blocklet from file, does not uncompress it.
-   * @param blocksChunkHolder
+   * @param rawBlockletColumnChunks
    */
-  void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException;
+  void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException;
 
-  /**
-   * In case if there is no filter satisfies.
-   * @return AbstractScannedResult
-   */
-  AbstractScannedResult createEmptyResult();
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
new file mode 100644
index 0000000..1c73d63
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.scan.scanner.impl;
+
+import java.io.IOException;
+import java.util.BitSet;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
+import org.apache.carbondata.core.scan.filter.executer.ImplicitColumnFilterExecutor;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
+import org.apache.carbondata.core.scan.result.impl.FilterQueryScannedResult;
+import org.apache.carbondata.core.stats.QueryStatistic;
+import org.apache.carbondata.core.stats.QueryStatisticsConstants;
+import org.apache.carbondata.core.stats.QueryStatisticsModel;
+import org.apache.carbondata.core.util.BitSetGroup;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+/**
+ * Below class will be used for filter query processing
+ * this class will be first apply the filter then it will read the column page if
+ * required and return the scanned result
+ */
+public class BlockletFilterScanner extends BlockletFullScanner {
+
+  /**
+   * filter executer to evaluate filter condition
+   */
+  private FilterExecuter filterExecuter;
+  /**
+   * this will be used to apply min max
+   * this will be useful for dimension column which is on the right side
+   * as node finder will always give tentative blocks, if column data stored individually
+   * and data is in sorted order then we can check whether filter is in the range of min max or not
+   * if it present then only we can apply filter on complete data.
+   * this will be very useful in case of sparse data when rows are
+   * repeating.
+   */
+  private boolean isMinMaxEnabled;
+
+  private QueryStatisticsModel queryStatisticsModel;
+
+  private boolean useBitSetPipeLine;
+
+  public BlockletFilterScanner(BlockExecutionInfo blockExecutionInfo,
+      QueryStatisticsModel queryStatisticsModel) {
+    super(blockExecutionInfo, queryStatisticsModel);
+    // to check whether min max is enabled or not
+    String minMaxEnableValue = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED,
+            CarbonCommonConstants.MIN_MAX_DEFAULT_VALUE);
+    if (null != minMaxEnableValue) {
+      isMinMaxEnabled = Boolean.parseBoolean(minMaxEnableValue);
+    }
+    // get the filter tree
+    this.filterExecuter = blockExecutionInfo.getFilterExecuterTree();
+    this.queryStatisticsModel = queryStatisticsModel;
+
+    String useBitSetPipeLine = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.BITSET_PIPE_LINE,
+            CarbonCommonConstants.BITSET_PIPE_LINE_DEFAULT);
+    if (null != useBitSetPipeLine) {
+      this.useBitSetPipeLine = Boolean.parseBoolean(useBitSetPipeLine);
+    }
+  }
+
+  /**
+   * Below method will be used to process the block
+   *
+   * @param rawBlockletColumnChunks block chunk holder which holds the data
+   * @throws FilterUnsupportedException
+   */
+  @Override
+  public BlockletScannedResult scanBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws IOException, FilterUnsupportedException {
+    return executeFilter(rawBlockletColumnChunks);
+  }
+
+  @Override
+  public boolean isScanRequired(DataRefNode dataBlock) {
+    // adding statistics for number of pages
+    QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
+    totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
+        totalPagesScanned.getCount() + dataBlock.numberOfPages());
+    // apply min max
+    if (isMinMaxEnabled) {
+      BitSet bitSet = null;
+      // check for implicit include filter instance
+      if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
+        String blockletId = blockExecutionInfo.getBlockIdString() +
+            CarbonCommonConstants.FILE_SEPARATOR + dataBlock.blockletIndex();
+        bitSet = ((ImplicitColumnFilterExecutor) filterExecuter)
+            .isFilterValuesPresentInBlockOrBlocklet(
+                dataBlock.getColumnsMaxValue(),
+                dataBlock.getColumnsMinValue(), blockletId);
+      } else {
+        bitSet = this.filterExecuter
+            .isScanRequired(dataBlock.getColumnsMaxValue(),
+                dataBlock.getColumnsMinValue());
+      }
+      return !bitSet.isEmpty();
+    }
+    return true;
+  }
+
+  @Override
+  public void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
+    long startTime = System.currentTimeMillis();
+    this.filterExecuter.readColumnChunks(rawBlockletColumnChunks);
+    // adding statistics for carbon read time
+    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
+    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
+        readTime.getCount() + (System.currentTimeMillis() - startTime));
+  }
+
+  /**
+   * This method will process the data in below order
+   * 1. first apply min max on the filter tree and check whether any of the filter
+   * is fall on the range of min max, if not then return empty result
+   * 2. If filter falls on min max range then apply filter on actual
+   * data and get the filtered row index
+   * 3. if row index is empty then return the empty result
+   * 4. if row indexes is not empty then read only those blocks(measure or dimension)
+   * which was present in the query but not present in the filter, as while applying filter
+   * some of the blocks where already read and present in chunk holder so not need to
+   * read those blocks again, this is to avoid reading of same blocks which was already read
+   * 5. Set the blocks and filter indexes to result
+   *
+   * @param rawBlockletColumnChunks
+   * @throws FilterUnsupportedException
+   */
+  private BlockletScannedResult executeFilter(RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws FilterUnsupportedException, IOException {
+    long startTime = System.currentTimeMillis();
+    QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
+    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
+        totalBlockletStatistic.getCount() + 1);
+    // apply filter on actual data, for each page
+    BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(rawBlockletColumnChunks,
+        useBitSetPipeLine);
+    // if filter result is empty then return with empty result
+    if (bitSetGroup.isEmpty()) {
+      CarbonUtil.freeMemory(rawBlockletColumnChunks.getDimensionRawColumnChunks(),
+          rawBlockletColumnChunks.getMeasureRawColumnChunks());
+
+      QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+          .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
+      scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
+          scanTime.getCount() + (System.currentTimeMillis() - startTime));
+
+      QueryStatistic scannedPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
+          .get(QueryStatisticsConstants.PAGE_SCANNED);
+      scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
+          scannedPages.getCount() + bitSetGroup.getScannedPages());
+      return createEmptyResult();
+    }
+
+    BlockletScannedResult scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
+    scannedResult.setBlockletId(
+        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR +
+            rawBlockletColumnChunks.getDataBlock().blockletIndex());
+    // valid scanned blocklet
+    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
+    validScannedBlockletStatistic
+        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
+            validScannedBlockletStatistic.getCount() + 1);
+    // adding statistics for valid number of pages
+    QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
+    validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
+        validPages.getCount() + bitSetGroup.getValidPages());
+    QueryStatistic scannedPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.PAGE_SCANNED);
+    scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
+        scannedPages.getCount() + bitSetGroup.getScannedPages());
+    int[] pageFilteredRowCount = new int[bitSetGroup.getNumberOfPages()];
+    // get the row indexes from bit set for each page
+    int[][] pageFilteredRowId = new int[bitSetGroup.getNumberOfPages()][];
+    int numPages = pageFilteredRowId.length;
+    for (int pageId = 0; pageId < numPages; pageId++) {
+      BitSet bitSet = bitSetGroup.getBitSet(pageId);
+      if (bitSet != null && !bitSet.isEmpty()) {
+        int[] matchedRowId = new int[bitSet.cardinality()];
+        int index = 0;
+        for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
+          matchedRowId[index++] = i;
+        }
+        pageFilteredRowCount[pageId] = matchedRowId.length;
+        pageFilteredRowId[pageId] = matchedRowId;
+      }
+    }
+
+    long dimensionReadTime = System.currentTimeMillis();
+    dimensionReadTime = System.currentTimeMillis() - dimensionReadTime;
+
+    FileReader fileReader = rawBlockletColumnChunks.getFileReader();
+
+
+    DimensionRawColumnChunk[] dimensionRawColumnChunks =
+        new DimensionRawColumnChunk[blockExecutionInfo.getTotalNumberDimensionToRead()];
+    int numDimensionChunks = dimensionRawColumnChunks.length;
+    // read dimension chunk blocks from file which is not present
+    for (int chunkIndex = 0; chunkIndex < numDimensionChunks; chunkIndex++) {
+      dimensionRawColumnChunks[chunkIndex] =
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
+    }
+    int[][] allSelectedDimensionColumnIndexRange =
+        blockExecutionInfo.getAllSelectedDimensionColumnIndexRange();
+    DimensionRawColumnChunk[] projectionListDimensionChunk = rawBlockletColumnChunks.getDataBlock()
+        .readDimensionChunks(fileReader, allSelectedDimensionColumnIndexRange);
+    for (int[] columnIndexRange : allSelectedDimensionColumnIndexRange) {
+      System.arraycopy(projectionListDimensionChunk, columnIndexRange[0],
+          dimensionRawColumnChunks, columnIndexRange[0],
+          columnIndexRange[1] + 1 - columnIndexRange[0]);
+    }
+
+    /*
+     * in case projection if the projected dimension are not loaded in the dimensionColumnDataChunk
+     * then loading them
+     */
+    int[] projectionListDimensionIndexes = blockExecutionInfo.getProjectionListDimensionIndexes();
+    for (int projectionListDimensionIndex : projectionListDimensionIndexes) {
+      if (null == dimensionRawColumnChunks[projectionListDimensionIndex]) {
+        dimensionRawColumnChunks[projectionListDimensionIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                fileReader, projectionListDimensionIndex);
+      }
+    }
+
+    DimensionColumnPage[][] dimensionColumnPages =
+        new DimensionColumnPage[numDimensionChunks][numPages];
+    for (int chunkIndex = 0; chunkIndex < numDimensionChunks; chunkIndex++) {
+      if (dimensionRawColumnChunks[chunkIndex] != null) {
+        for (int pageId = 0; pageId < numPages; pageId++) {
+          dimensionColumnPages[chunkIndex][pageId] =
+              dimensionRawColumnChunks[chunkIndex].decodeColumnPage(pageId);
+        }
+      }
+    }
+
+
+    MeasureRawColumnChunk[] measureRawColumnChunks =
+        new MeasureRawColumnChunk[blockExecutionInfo.getTotalNumberOfMeasureToRead()];
+    int numMeasureChunks = measureRawColumnChunks.length;
+
+    // read the measure chunk blocks which is not present
+    for (int chunkIndex = 0; chunkIndex < numMeasureChunks; chunkIndex++) {
+      if (null != rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        measureRawColumnChunks[chunkIndex] =
+            rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
+      }
+    }
+
+    int[][] allSelectedMeasureColumnIndexRange =
+        blockExecutionInfo.getAllSelectedMeasureIndexRange();
+    MeasureRawColumnChunk[] projectionListMeasureChunk = rawBlockletColumnChunks.getDataBlock()
+        .readMeasureChunks(fileReader, allSelectedMeasureColumnIndexRange);
+    for (int[] columnIndexRange : allSelectedMeasureColumnIndexRange) {
+      System.arraycopy(projectionListMeasureChunk, columnIndexRange[0], measureRawColumnChunks,
+          columnIndexRange[0], columnIndexRange[1] + 1 - columnIndexRange[0]);
+    }
+    /*
+     * in case projection if the projected measure are not loaded in the ColumnPage
+     * then loading them
+     */
+    int[] projectionListMeasureIndexes = blockExecutionInfo.getProjectionListMeasureIndexes();
+    for (int projectionListMeasureIndex : projectionListMeasureIndexes) {
+      if (null == measureRawColumnChunks[projectionListMeasureIndex]) {
+        measureRawColumnChunks[projectionListMeasureIndex] = rawBlockletColumnChunks.getDataBlock()
+            .readMeasureChunk(fileReader, projectionListMeasureIndex);
+      }
+    }
+    ColumnPage[][] measureColumnPages = new ColumnPage[numMeasureChunks][numPages];
+    for (int chunkIndex = 0; chunkIndex < numMeasureChunks; chunkIndex++) {
+      if (measureRawColumnChunks[chunkIndex] != null) {
+        for (int pageId = 0; pageId < numPages; pageId++) {
+          measureColumnPages[chunkIndex][pageId] =
+              measureRawColumnChunks[chunkIndex].decodeColumnPage(pageId);
+        }
+      }
+    }
+
+    scannedResult.setDimensionColumnPages(dimensionColumnPages);
+    scannedResult.setPageFilteredRowId(pageFilteredRowId);
+    scannedResult.setMeasureColumnPages(measureColumnPages);
+    scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks);
+    scannedResult.setMsrRawColumnChunks(measureRawColumnChunks);
+    scannedResult.setPageFilteredRowCount(pageFilteredRowCount);
+    // adding statistics for carbon scan time
+    QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
+    scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
+        scanTime.getCount() + (System.currentTimeMillis() - startTime - dimensionReadTime));
+    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
+    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
+        readTime.getCount() + dimensionReadTime);
+    return scannedResult;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java
new file mode 100644
index 0000000..f0211dc
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFullScanner.java
@@ -0,0 +1,191 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.scanner.impl;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
+import org.apache.carbondata.core.scan.result.impl.NonFilterQueryScannedResult;
+import org.apache.carbondata.core.scan.scanner.BlockletScanner;
+import org.apache.carbondata.core.stats.QueryStatistic;
+import org.apache.carbondata.core.stats.QueryStatisticsConstants;
+import org.apache.carbondata.core.stats.QueryStatisticsModel;
+
+/**
+ * Blocklet scanner to do full scan of a blocklet,
+ * returning all projection and filter column chunks
+ */
+public class BlockletFullScanner implements BlockletScanner {
+
+  /**
+   * block execution info
+   */
+  protected BlockExecutionInfo blockExecutionInfo;
+
+  private QueryStatisticsModel queryStatisticsModel;
+
+  private BlockletScannedResult emptyResult;
+
+  public BlockletFullScanner(BlockExecutionInfo tableBlockExecutionInfos,
+      QueryStatisticsModel queryStatisticsModel) {
+    this.blockExecutionInfo = tableBlockExecutionInfos;
+    this.queryStatisticsModel = queryStatisticsModel;
+  }
+
+  @Override
+  public BlockletScannedResult scanBlocklet(
+      RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws IOException, FilterUnsupportedException {
+    long startTime = System.currentTimeMillis();
+    BlockletScannedResult scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
+    QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
+    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
+        totalBlockletStatistic.getCount() + 1);
+    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
+    validScannedBlockletStatistic
+        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
+            validScannedBlockletStatistic.getCount() + 1);
+    // adding statistics for valid number of pages
+    QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
+    validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
+        validPages.getCount() + rawBlockletColumnChunks.getDataBlock().numberOfPages());
+    // adding statistics for number of pages
+    QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
+    totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
+        totalPagesScanned.getCount() + rawBlockletColumnChunks.getDataBlock().numberOfPages());
+    scannedResult.setBlockletId(
+        blockExecutionInfo.getBlockIdString() + CarbonCommonConstants.FILE_SEPARATOR +
+            rawBlockletColumnChunks.getDataBlock().blockletIndex());
+    if (!blockExecutionInfo.isPrefetchBlocklet()) {
+      readBlocklet(rawBlockletColumnChunks);
+    }
+    DimensionRawColumnChunk[] dimensionRawColumnChunks =
+        rawBlockletColumnChunks.getDimensionRawColumnChunks();
+    DimensionColumnPage[][] dimensionColumnDataChunks =
+        new DimensionColumnPage[dimensionRawColumnChunks.length][rawBlockletColumnChunks
+            .getDataBlock().numberOfPages()];
+    MeasureRawColumnChunk[] measureRawColumnChunks =
+        rawBlockletColumnChunks.getMeasureRawColumnChunks();
+    ColumnPage[][] measureColumnPages =
+        new ColumnPage[measureRawColumnChunks.length][rawBlockletColumnChunks.getDataBlock()
+                       .numberOfPages()];
+    scannedResult.setDimensionColumnPages(dimensionColumnDataChunks);
+    scannedResult.setMeasureColumnPages(measureColumnPages);
+    scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks);
+    scannedResult.setMsrRawColumnChunks(measureRawColumnChunks);
+    if (blockExecutionInfo.isPrefetchBlocklet()) {
+      for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
+        if (dimensionRawColumnChunks[i] != null) {
+          dimensionColumnDataChunks[i] = dimensionRawColumnChunks[i].decodeAllColumnPages();
+        }
+      }
+      for (int i = 0; i < measureRawColumnChunks.length; i++) {
+        if (measureRawColumnChunks[i] != null) {
+          measureColumnPages[i] = measureRawColumnChunks[i].decodeAllColumnPages();
+        }
+      }
+    }
+    int[] numberOfRows = null;
+    if (blockExecutionInfo.getAllSelectedDimensionColumnIndexRange().length > 0) {
+      for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
+        if (dimensionRawColumnChunks[i] != null) {
+          numberOfRows = dimensionRawColumnChunks[i].getRowCount();
+          break;
+        }
+      }
+    } else if (blockExecutionInfo.getAllSelectedMeasureIndexRange().length > 0) {
+      for (int i = 0; i < measureRawColumnChunks.length; i++) {
+        if (measureRawColumnChunks[i] != null) {
+          numberOfRows = measureRawColumnChunks[i].getRowCount();
+          break;
+        }
+      }
+    }
+
+    // count(*)  case there would not be any dimensions are measures selected.
+    if (numberOfRows == null) {
+      numberOfRows = new int[rawBlockletColumnChunks.getDataBlock().numberOfPages()];
+      for (int i = 0; i < numberOfRows.length; i++) {
+        numberOfRows[i] =
+            CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      }
+      int lastPageSize = rawBlockletColumnChunks.getDataBlock().numRows()
+          % CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT;
+      ;
+      if (lastPageSize > 0) {
+        numberOfRows[numberOfRows.length - 1] = lastPageSize;
+      }
+    }
+    scannedResult.setPageFilteredRowCount(numberOfRows);
+    if (!blockExecutionInfo.isPrefetchBlocklet()) {
+      scannedResult.fillDataChunks();
+    }
+    // adding statistics for carbon scan time
+    QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
+    scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
+        scanTime.getCount() + (System.currentTimeMillis() - startTime));
+    return scannedResult;
+  }
+
+  @Override
+  public void readBlocklet(RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws IOException {
+    long startTime = System.currentTimeMillis();
+    DimensionRawColumnChunk[] dimensionRawColumnChunks = rawBlockletColumnChunks.getDataBlock()
+        .readDimensionChunks(rawBlockletColumnChunks.getFileReader(),
+            blockExecutionInfo.getAllSelectedDimensionColumnIndexRange());
+    rawBlockletColumnChunks.setDimensionRawColumnChunks(dimensionRawColumnChunks);
+    MeasureRawColumnChunk[] measureRawColumnChunks = rawBlockletColumnChunks.getDataBlock()
+        .readMeasureChunks(rawBlockletColumnChunks.getFileReader(),
+            blockExecutionInfo.getAllSelectedMeasureIndexRange());
+    rawBlockletColumnChunks.setMeasureRawColumnChunks(measureRawColumnChunks);
+    // adding statistics for carbon read time
+    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
+        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
+    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
+        readTime.getCount() + (System.currentTimeMillis() - startTime));
+  }
+
+  BlockletScannedResult createEmptyResult() {
+    if (emptyResult == null) {
+      emptyResult = new NonFilterQueryScannedResult(blockExecutionInfo);
+      emptyResult.setPageFilteredRowCount(new int[0]);
+      emptyResult.setPageFilteredRowId(new int[0][]);
+    }
+    return emptyResult;
+  }
+
+  @Override public boolean isScanRequired(DataRefNode dataBlock) {
+    // For non filter it is always true
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java
deleted file mode 100644
index e77093b..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/FilterScanner.java
+++ /dev/null
@@ -1,326 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.scanner.impl;
-
-import java.io.IOException;
-import java.util.BitSet;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.datastore.page.ColumnPage;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
-import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
-import org.apache.carbondata.core.scan.filter.executer.ImplicitColumnFilterExecutor;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
-import org.apache.carbondata.core.scan.result.impl.FilterQueryScannedResult;
-import org.apache.carbondata.core.scan.scanner.AbstractBlockletScanner;
-import org.apache.carbondata.core.stats.QueryStatistic;
-import org.apache.carbondata.core.stats.QueryStatisticsConstants;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-import org.apache.carbondata.core.util.BitSetGroup;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-/**
- * Below class will be used for filter query processing
- * this class will be first apply the filter then it will read the block if
- * required and return the scanned result
- */
-public class FilterScanner extends AbstractBlockletScanner {
-
-  /**
-   * filter tree
-   */
-  private FilterExecuter filterExecuter;
-  /**
-   * this will be used to apply min max
-   * this will be useful for dimension column which is on the right side
-   * as node finder will always give tentative blocks, if column data stored individually
-   * and data is in sorted order then we can check whether filter is in the range of min max or not
-   * if it present then only we can apply filter on complete data.
-   * this will be very useful in case of sparse data when rows are
-   * repeating.
-   */
-  private boolean isMinMaxEnabled;
-
-  private QueryStatisticsModel queryStatisticsModel;
-
-  private boolean useBitSetPipeLine;
-
-  public FilterScanner(BlockExecutionInfo blockExecutionInfo,
-      QueryStatisticsModel queryStatisticsModel) {
-    super(blockExecutionInfo);
-    // to check whether min max is enabled or not
-    String minMaxEnableValue = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED,
-            CarbonCommonConstants.MIN_MAX_DEFAULT_VALUE);
-    if (null != minMaxEnableValue) {
-      isMinMaxEnabled = Boolean.parseBoolean(minMaxEnableValue);
-    }
-    // get the filter tree
-    this.filterExecuter = blockExecutionInfo.getFilterExecuterTree();
-    this.queryStatisticsModel = queryStatisticsModel;
-
-    String useBitSetPipeLine = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BITSET_PIPE_LINE,
-            CarbonCommonConstants.BITSET_PIPE_LINE_DEFAULT);
-    if (null != useBitSetPipeLine) {
-      this.useBitSetPipeLine = Boolean.parseBoolean(useBitSetPipeLine);
-    }
-  }
-
-  /**
-   * Below method will be used to process the block
-   *
-   * @param blocksChunkHolder block chunk holder which holds the data
-   * @throws FilterUnsupportedException
-   */
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws IOException, FilterUnsupportedException {
-    return fillScannedResult(blocksChunkHolder);
-  }
-
-  @Override public boolean isScanRequired(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    // adding statistics for number of pages
-    QueryStatistic totalPagesScanned = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_PAGE_SCANNED);
-    totalPagesScanned.addCountStatistic(QueryStatisticsConstants.TOTAL_PAGE_SCANNED,
-        totalPagesScanned.getCount() + blocksChunkHolder.getDataBlock().numberOfPages());
-    // apply min max
-    if (isMinMaxEnabled) {
-      BitSet bitSet = null;
-      // check for implicit include filter instance
-      if (filterExecuter instanceof ImplicitColumnFilterExecutor) {
-        String blockletId = blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR
-            + blocksChunkHolder.getDataBlock().blockletId();
-        bitSet = ((ImplicitColumnFilterExecutor) filterExecuter)
-            .isFilterValuesPresentInBlockOrBlocklet(
-                blocksChunkHolder.getDataBlock().getColumnsMaxValue(),
-                blocksChunkHolder.getDataBlock().getColumnsMinValue(), blockletId);
-      } else {
-        bitSet = this.filterExecuter
-            .isScanRequired(blocksChunkHolder.getDataBlock().getColumnsMaxValue(),
-                blocksChunkHolder.getDataBlock().getColumnsMinValue());
-      }
-      if (bitSet.isEmpty()) {
-        CarbonUtil.freeMemory(blocksChunkHolder.getDimensionRawDataChunk(),
-            blocksChunkHolder.getMeasureRawDataChunk());
-        return false;
-      }
-    }
-    return true;
-  }
-
-  @Override public void readBlocklet(BlocksChunkHolder blocksChunkHolder) throws IOException {
-    long startTime = System.currentTimeMillis();
-    this.filterExecuter.readBlocks(blocksChunkHolder);
-    // adding statistics for carbon read time
-    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
-    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
-        readTime.getCount() + (System.currentTimeMillis() - startTime));
-  }
-
-  /**
-   * This method will process the data in below order
-   * 1. first apply min max on the filter tree and check whether any of the filter
-   * is fall on the range of min max, if not then return empty result
-   * 2. If filter falls on min max range then apply filter on actual
-   * data and get the filtered row index
-   * 3. if row index is empty then return the empty result
-   * 4. if row indexes is not empty then read only those blocks(measure or dimension)
-   * which was present in the query but not present in the filter, as while applying filter
-   * some of the blocks where already read and present in chunk holder so not need to
-   * read those blocks again, this is to avoid reading of same blocks which was already read
-   * 5. Set the blocks and filter indexes to result
-   *
-   * @param blocksChunkHolder
-   * @throws FilterUnsupportedException
-   */
-  private AbstractScannedResult fillScannedResult(BlocksChunkHolder blocksChunkHolder)
-      throws FilterUnsupportedException, IOException {
-    long startTime = System.currentTimeMillis();
-    QueryStatistic totalBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
-    totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
-        totalBlockletStatistic.getCount() + 1);
-    // apply filter on actual data
-    BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(blocksChunkHolder, useBitSetPipeLine);
-    // if indexes is empty then return with empty result
-    if (bitSetGroup.isEmpty()) {
-      CarbonUtil.freeMemory(blocksChunkHolder.getDimensionRawDataChunk(),
-          blocksChunkHolder.getMeasureRawDataChunk());
-
-      QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-          .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
-      scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
-          scanTime.getCount() + (System.currentTimeMillis() - startTime));
-
-      QueryStatistic scannedPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
-          .get(QueryStatisticsConstants.PAGE_SCANNED);
-      scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
-          scannedPages.getCount() + bitSetGroup.getScannedPages());
-      return createEmptyResult();
-    }
-
-    AbstractScannedResult scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
-    scannedResult.setBlockletId(
-        blockExecutionInfo.getBlockId() + CarbonCommonConstants.FILE_SEPARATOR + blocksChunkHolder
-            .getDataBlock().blockletId());
-    // valid scanned blocklet
-    QueryStatistic validScannedBlockletStatistic = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM);
-    validScannedBlockletStatistic
-        .addCountStatistic(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM,
-            validScannedBlockletStatistic.getCount() + 1);
-    // adding statistics for valid number of pages
-    QueryStatistic validPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.VALID_PAGE_SCANNED);
-    validPages.addCountStatistic(QueryStatisticsConstants.VALID_PAGE_SCANNED,
-        validPages.getCount() + bitSetGroup.getValidPages());
-    QueryStatistic scannedPages = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.PAGE_SCANNED);
-    scannedPages.addCountStatistic(QueryStatisticsConstants.PAGE_SCANNED,
-        scannedPages.getCount() + bitSetGroup.getScannedPages());
-    int[] rowCount = new int[bitSetGroup.getNumberOfPages()];
-    // get the row indexes from bot set
-    int[][] indexesGroup = new int[bitSetGroup.getNumberOfPages()][];
-    for (int k = 0; k < indexesGroup.length; k++) {
-      BitSet bitSet = bitSetGroup.getBitSet(k);
-      if (bitSet != null && !bitSet.isEmpty()) {
-        int[] indexes = new int[bitSet.cardinality()];
-        int index = 0;
-        for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-          indexes[index++] = i;
-        }
-        rowCount[k] = indexes.length;
-        indexesGroup[k] = indexes;
-      }
-    }
-    FileHolder fileReader = blocksChunkHolder.getFileReader();
-    int[][] allSelectedDimensionBlocksIndexes =
-        blockExecutionInfo.getAllSelectedDimensionBlocksIndexes();
-    long dimensionReadTime = System.currentTimeMillis();
-    DimensionRawColumnChunk[] projectionListDimensionChunk = blocksChunkHolder.getDataBlock()
-        .getDimensionChunks(fileReader, allSelectedDimensionBlocksIndexes);
-    dimensionReadTime = System.currentTimeMillis() - dimensionReadTime;
-
-    DimensionRawColumnChunk[] dimensionRawColumnChunks =
-        new DimensionRawColumnChunk[blockExecutionInfo.getTotalNumberDimensionBlock()];
-    // read dimension chunk blocks from file which is not present
-    for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
-      if (null != blocksChunkHolder.getDimensionRawDataChunk()[i]) {
-        dimensionRawColumnChunks[i] = blocksChunkHolder.getDimensionRawDataChunk()[i];
-      }
-    }
-    for (int i = 0; i < allSelectedDimensionBlocksIndexes.length; i++) {
-      for (int j = allSelectedDimensionBlocksIndexes[i][0];
-           j <= allSelectedDimensionBlocksIndexes[i][1]; j++) {
-        dimensionRawColumnChunks[j] = projectionListDimensionChunk[j];
-      }
-    }
-    long dimensionReadTime1 = System.currentTimeMillis();
-    /**
-     * in case projection if the projected dimension are not loaded in the dimensionColumnDataChunk
-     * then loading them
-     */
-    int[] projectionListDimensionIndexes = blockExecutionInfo.getProjectionListDimensionIndexes();
-    int projectionListDimensionIndexesLength = projectionListDimensionIndexes.length;
-    for (int i = 0; i < projectionListDimensionIndexesLength; i++) {
-      if (null == dimensionRawColumnChunks[projectionListDimensionIndexes[i]]) {
-        dimensionRawColumnChunks[projectionListDimensionIndexes[i]] =
-            blocksChunkHolder.getDataBlock()
-                .getDimensionChunk(fileReader, projectionListDimensionIndexes[i]);
-      }
-    }
-    dimensionReadTime += (System.currentTimeMillis() - dimensionReadTime1);
-    dimensionReadTime1 = System.currentTimeMillis();
-    MeasureRawColumnChunk[] measureRawColumnChunks =
-        new MeasureRawColumnChunk[blockExecutionInfo.getTotalNumberOfMeasureBlock()];
-    int[][] allSelectedMeasureBlocksIndexes =
-        blockExecutionInfo.getAllSelectedMeasureBlocksIndexes();
-    MeasureRawColumnChunk[] projectionListMeasureChunk = blocksChunkHolder.getDataBlock()
-        .getMeasureChunks(fileReader, allSelectedMeasureBlocksIndexes);
-    dimensionReadTime += System.currentTimeMillis() - dimensionReadTime1;
-    // read the measure chunk blocks which is not present
-    for (int i = 0; i < measureRawColumnChunks.length; i++) {
-      if (null != blocksChunkHolder.getMeasureRawDataChunk()[i]) {
-        measureRawColumnChunks[i] = blocksChunkHolder.getMeasureRawDataChunk()[i];
-      }
-    }
-    for (int i = 0; i < allSelectedMeasureBlocksIndexes.length; i++) {
-      for (int j = allSelectedMeasureBlocksIndexes[i][0];
-           j <= allSelectedMeasureBlocksIndexes[i][1]; j++) {
-        measureRawColumnChunks[j] = projectionListMeasureChunk[j];
-      }
-    }
-    dimensionReadTime1 = System.currentTimeMillis();
-    /**
-     * in case projection if the projected measure are not loaded in the ColumnPage
-     * then loading them
-     */
-    int[] projectionListMeasureIndexes = blockExecutionInfo.getProjectionListMeasureIndexes();
-    int projectionListMeasureIndexesLength = projectionListMeasureIndexes.length;
-    for (int i = 0; i < projectionListMeasureIndexesLength; i++) {
-      if (null == measureRawColumnChunks[projectionListMeasureIndexes[i]]) {
-        measureRawColumnChunks[projectionListMeasureIndexes[i]] = blocksChunkHolder.getDataBlock()
-            .getMeasureChunk(fileReader, projectionListMeasureIndexes[i]);
-      }
-    }
-    dimensionReadTime += System.currentTimeMillis() - dimensionReadTime1;
-    DimensionColumnDataChunk[][] dimensionColumnDataChunks =
-        new DimensionColumnDataChunk[dimensionRawColumnChunks.length][indexesGroup.length];
-    ColumnPage[][] columnPages =
-        new ColumnPage[measureRawColumnChunks.length][indexesGroup.length];
-    for (int i = 0; i < dimensionRawColumnChunks.length; i++) {
-      if (dimensionRawColumnChunks[i] != null) {
-        for (int j = 0; j < indexesGroup.length; j++) {
-          dimensionColumnDataChunks[i][j] = dimensionRawColumnChunks[i].convertToDimColDataChunk(j);
-        }
-      }
-    }
-    for (int i = 0; i < measureRawColumnChunks.length; i++) {
-      if (measureRawColumnChunks[i] != null) {
-        for (int j = 0; j < indexesGroup.length; j++) {
-          columnPages[i][j] = measureRawColumnChunks[i].convertToColumnPage(j);
-        }
-      }
-    }
-    scannedResult.setDimensionChunks(dimensionColumnDataChunks);
-    scannedResult.setIndexes(indexesGroup);
-    scannedResult.setMeasureChunks(columnPages);
-    scannedResult.setDimRawColumnChunks(dimensionRawColumnChunks);
-    scannedResult.setMsrRawColumnChunks(measureRawColumnChunks);
-    scannedResult.setNumberOfRows(rowCount);
-    // adding statistics for carbon scan time
-    QueryStatistic scanTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.SCAN_BLOCKlET_TIME);
-    scanTime.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKlET_TIME,
-        scanTime.getCount() + (System.currentTimeMillis() - startTime - dimensionReadTime));
-    QueryStatistic readTime = queryStatisticsModel.getStatisticsTypeAndObjMap()
-        .get(QueryStatisticsConstants.READ_BLOCKlET_TIME);
-    readTime.addCountStatistic(QueryStatisticsConstants.READ_BLOCKlET_TIME,
-        readTime.getCount() + dimensionReadTime);
-    return scannedResult;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java
deleted file mode 100644
index 1373ed5..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/NonFilterScanner.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.scanner.impl;
-
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.scanner.AbstractBlockletScanner;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-
-/**
- * Non filter processor which will be used for non filter query
- * In case of non filter query we just need to read all the blocks requested in the
- * query and pass it to scanned result
- */
-public class NonFilterScanner extends AbstractBlockletScanner {
-
-  public NonFilterScanner(BlockExecutionInfo blockExecutionInfo,
-                          QueryStatisticsModel queryStatisticsModel) {
-    super(blockExecutionInfo);
-    super.queryStatisticsModel = queryStatisticsModel;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
index 2f981b5..6faae03 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
@@ -30,22 +30,17 @@ public class ByteArrayWrapper implements Comparable<ByteArrayWrapper>, Serializa
    * to store key which is generated using
    * key generator
    */
-  protected byte[] dictionaryKey;
+  private byte[] dictionaryKey;
 
   /**
    * to store no dictionary column data
    */
-  protected byte[][] complexTypesKeys;
+  private byte[][] complexTypesKeys;
 
   /**
    * to store no dictionary column data
    */
-  protected byte[][] noDictionaryKeys;
-
-  /**
-   * contains value of implicit columns in byte array format
-   */
-  protected byte[] implicitColumnByteArray;
+  private byte[][] noDictionaryKeys;
 
   public ByteArrayWrapper() {
   }
@@ -91,16 +86,6 @@ public class ByteArrayWrapper implements Comparable<ByteArrayWrapper>, Serializa
   }
 
   /**
-   * to get the no dictionary column data
-   *
-   * @param index of the no dictionary key
-   * @return no dictionary key for the index
-   */
-  public byte[] getComplexTypeByIndex(int index) {
-    return this.complexTypesKeys[index];
-  }
-
-  /**
    * to generate the hash code
    */
   @Override public int hashCode() {
@@ -201,30 +186,10 @@ public class ByteArrayWrapper implements Comparable<ByteArrayWrapper>, Serializa
   }
 
   /**
-   * @return the complexTypesKeys
-   */
-  public byte[][] getComplexTypesKeys() {
-    return complexTypesKeys;
-  }
-
-  /**
    * @param complexTypesKeys the complexTypesKeys to set
    */
   public void setComplexTypesKeys(byte[][] complexTypesKeys) {
     this.complexTypesKeys = complexTypesKeys;
   }
 
-  /**
-   * @return
-   */
-  public byte[] getImplicitColumnByteArray() {
-    return implicitColumnByteArray;
-  }
-
-  /**
-   * @param implicitColumnByteArray
-   */
-  public void setImplicitColumnByteArray(byte[] implicitColumnByteArray) {
-    this.implicitColumnByteArray = implicitColumnByteArray;
-  }
 }

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java b/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java
index 55f0882..ed60d37 100644
--- a/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/stats/QueryStatisticsModel.java
@@ -37,7 +37,4 @@ public class QueryStatisticsModel {
     return statisticsTypeAndObjMap;
   }
 
-  public void setStatisticsTypeAndObjMap(Map<String, QueryStatistic> statisticsTypeAndObjMap) {
-    this.statisticsTypeAndObjMap = statisticsTypeAndObjMap;
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
index b6a9e36..d6671b4 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
@@ -345,15 +345,6 @@ public class LoadMetadataDetails implements Serializable {
   }
 
   /**
-   * To get isDeleted property.
-   *
-   * @return isDeleted
-   */
-  public String getIsDeleted() {
-    return isDeleted;
-  }
-
-  /**
    * To set isDeleted property.
    *
    * @param isDeleted

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index 76c2dc7..3c991e0 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -680,22 +680,6 @@ public class SegmentStatusManager {
     return "";
   }
 
-  /**
-   * getting the task numbers present in the segment.
-   * @param segmentId
-   * @return
-   */
-  public List<String> getUpdatedTasksDetailsForSegment(String segmentId, SegmentUpdateStatusManager
-          updateStatusManager) {
-    List<String> taskList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<String> list = updateStatusManager.getUpdateDeltaFiles(segmentId);
-    for (String eachFileName : list) {
-      taskList.add(CarbonTablePath.DataFileUtil.getTaskNo(eachFileName));
-    }
-    return taskList;
-  }
-
-
   public static class ValidAndInvalidSegmentsInfo {
     private final List<Segment> listOfValidSegments;
     private final List<Segment> listOfValidUpdatedSegments;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
index 2edb379..019a20c 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
@@ -48,7 +48,6 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
 import org.apache.carbondata.core.mutate.TupleIdEnum;
 import org.apache.carbondata.core.mutate.UpdateVO;
-import org.apache.carbondata.core.reader.CarbonDeleteFilesDataReader;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
@@ -145,14 +144,6 @@ public class SegmentUpdateStatusManager {
   }
 
   /**
-   *
-   * @param loadMetadataDetails
-   */
-  public void setLoadMetadataDetails(LoadMetadataDetails[] loadMetadataDetails) {
-    this.segmentDetails = loadMetadataDetails;
-  }
-
-  /**
    * Returns the UpdateStatus Details.
    * @return
    */
@@ -179,18 +170,6 @@ public class SegmentUpdateStatusManager {
   }
 
   /**
-   * Returns all delete delta files of specified block
-   *
-   * @param tupleId
-   * @return
-   * @throws Exception
-   */
-  public List<String> getDeleteDeltaFiles(String tupleId) throws Exception {
-    return getDeltaFiles(tupleId, CarbonCommonConstants.DELETE_DELTA_FILE_EXT);
-  }
-
-
-  /**
    * Returns all update delta files of specified Segment.
    *
    * @param segmentId
@@ -253,20 +232,6 @@ public class SegmentUpdateStatusManager {
   }
 
   /**
-   * Returns all deleted records of specified block
-   *
-   * @param tupleId
-   * @return
-   * @throws Exception
-   */
-  public Map<Integer, Integer[]> getDeleteDeltaDataFromAllFiles(String tupleId) throws Exception {
-    List<String> deltaFiles = getDeltaFiles(tupleId, CarbonCommonConstants.DELETE_DELTA_FILE_EXT);
-    CarbonDeleteFilesDataReader dataReader = new CarbonDeleteFilesDataReader();
-    String blockletId = CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.BLOCKLET_ID);
-    return dataReader.getDeleteDataFromAllFiles(deltaFiles, blockletId);
-  }
-
-  /**
    * Below method will be used to get all the delete delta files based on block name
    *
    * @param blockFilePath actual block filePath
@@ -788,41 +753,6 @@ public class SegmentUpdateStatusManager {
   }
 
   /**
-   * compares passed time stamp with status file delete timestamp and
-   * returns latest timestamp from status file if both are not equal
-   * returns null otherwise
-   *
-   * @param completeBlockName
-   * @param timestamp
-   * @return
-   */
-  public String getTimestampForRefreshCache(String completeBlockName, String timestamp) {
-    long cacheTimestamp = 0;
-    if (null != timestamp) {
-      cacheTimestamp = CarbonUpdateUtil.getTimeStampAsLong(timestamp);
-    }
-    String blockName = CarbonTablePath.addDataPartPrefix(CarbonUpdateUtil.getBlockName(
-        CarbonUpdateUtil.getRequiredFieldFromTID(completeBlockName, TupleIdEnum.BLOCK_ID)));
-    String segmentId =
-        CarbonUpdateUtil.getRequiredFieldFromTID(completeBlockName, TupleIdEnum.SEGMENT_ID);
-    SegmentUpdateDetails[] listOfSegmentUpdateDetailsArray =
-        readLoadMetadata();
-    for (SegmentUpdateDetails block : listOfSegmentUpdateDetailsArray) {
-      if (segmentId.equalsIgnoreCase(block.getSegmentName()) &&
-          block.getBlockName().equalsIgnoreCase(blockName) &&
-          !CarbonUpdateUtil.isBlockInvalid(block.getSegmentStatus())) {
-        long deleteTimestampFromStatusFile = block.getDeleteDeltaEndTimeAsLong();
-        if (Long.compare(deleteTimestampFromStatusFile, cacheTimestamp) == 0) {
-          return null;
-        } else {
-          return block.getDeleteDeltaEndTimestamp();
-        }
-      }
-    }
-    return null;
-  }
-
-  /**
    * This method closes the streams
    *
    * @param streams - streams to close.
@@ -841,85 +771,7 @@ public class SegmentUpdateStatusManager {
       }
     }
   }
-  /**
-   * Get the invalid tasks in that segment.
-   * @param segmentId
-   * @return
-   */
-  public List<String> getInvalidBlockList(String segmentId) {
-
-    // get the original fact file timestamp from the table status file.
-    List<String> listOfInvalidBlocks = new ArrayList<String>();
-    SegmentStatusManager ssm = new SegmentStatusManager(absoluteTableIdentifier);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    LoadMetadataDetails[] segmentDetails =
-        ssm.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
-    long timestampOfOriginalFacts = 0;
-
-    String startTimestampOfUpdate = "" ;
-    String endTimestampOfUpdate = "";
-
-    for (LoadMetadataDetails segment : segmentDetails) {
-      // find matching segment and return timestamp.
-      if (segment.getLoadName().equalsIgnoreCase(segmentId)) {
-        timestampOfOriginalFacts = segment.getLoadStartTime();
-        startTimestampOfUpdate = segment.getUpdateDeltaStartTimestamp();
-        endTimestampOfUpdate = segment.getUpdateDeltaEndTimestamp();
-      }
-    }
-
-    if (startTimestampOfUpdate.isEmpty()) {
-      return listOfInvalidBlocks;
-
-    }
-
-    // now after getting the original fact timestamp, what ever is remaining
-    // files need to cross check it with table status file.
-
-    // filter out the fact files.
-
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
-    CarbonFile segDir =
-        FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
-
-    final Long endTimeStampFinal = CarbonUpdateUtil.getTimeStampAsLong(endTimestampOfUpdate);
-    final Long startTimeStampFinal = CarbonUpdateUtil.getTimeStampAsLong(startTimestampOfUpdate);
-    final Long timeStampOriginalFactFinal =
-        timestampOfOriginalFacts;
-
-    CarbonFile[] files = segDir.listFiles(new CarbonFileFilter() {
-
-      @Override public boolean accept(CarbonFile pathName) {
-        String fileName = pathName.getName();
-        if (fileName.endsWith(CarbonCommonConstants.UPDATE_DELTA_FILE_EXT)) {
-          String firstPart = fileName.substring(0, fileName.indexOf('.'));
-
-          long timestamp = Long.parseLong(firstPart
-              .substring(firstPart.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1,
-                  firstPart.length()));
-          if (Long.compare(timestamp, endTimeStampFinal) <= 0
-              && Long.compare(timestamp, startTimeStampFinal) >= 0) {
-            return false;
-          }
-          if (Long.compare(timestamp, timeStampOriginalFactFinal) == 0) {
-            return false;
-          }
-          // take the rest of files as they are invalid.
-          return true;
-        }
-        return false;
-      }
-    });
 
-    // gather the task numbers.
-    for (CarbonFile updateFiles : files) {
-      listOfInvalidBlocks.add(updateFiles.getName());
-    }
-
-    return listOfInvalidBlocks;
-  }
   /**
    * Returns the invalid timestamp range of a segment.
    * @param segmentId
@@ -945,12 +797,11 @@ public class SegmentUpdateStatusManager {
   }
   /**
    *
-   * @param segmentId
    * @param block
    * @param needCompleteList
    * @return
    */
-  public CarbonFile[] getDeleteDeltaInvalidFilesList(final String segmentId,
+  public CarbonFile[] getDeleteDeltaInvalidFilesList(
       final SegmentUpdateDetails block, final boolean needCompleteList,
       CarbonFile[] allSegmentFiles, boolean isAbortedFile) {
 
@@ -996,12 +847,11 @@ public class SegmentUpdateStatusManager {
 
   /**
    *
-   * @param blockName
    * @param allSegmentFiles
    * @return
    */
-  public CarbonFile[] getAllBlockRelatedFiles(String blockName, CarbonFile[] allSegmentFiles,
-                                              String actualBlockName) {
+  public CarbonFile[] getAllBlockRelatedFiles(CarbonFile[] allSegmentFiles,
+      String actualBlockName) {
     List<CarbonFile> files = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
     for (CarbonFile eachFile : allSegmentFiles) {


[36/49] carbondata git commit: [CARBONDATA-2156] Add interface annotation

Posted by ja...@apache.org.
[CARBONDATA-2156] Add interface annotation

InterfaceAudience and InterfaceStability annotation should be added for user and developer

1.InetfaceAudience can be User and Developer
2.InterfaceStability can be Stable, Evolving, Unstable

This closes #1968


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

Branch: refs/heads/carbonstore-rebase4
Commit: 97139299f7fb4a38bb851bbd5e9e4583089be15d
Parents: 2058a47
Author: Jacky Li <ja...@qq.com>
Authored: Sun Feb 11 10:12:10 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:09:07 2018 +0800

----------------------------------------------------------------------
 .../common/annotations/InterfaceAudience.java   | 58 ++++++++++++++++
 .../common/annotations/InterfaceStability.java  | 69 ++++++++++++++++++++
 2 files changed, 127 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/97139299/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
new file mode 100644
index 0000000..fa9729d
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.annotations;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * This annotation is ported and modified from Apache Hadoop project.
+ *
+ * Annotation to inform users of a package, class or method's intended audience.
+ * Currently the audience can be {@link User}, {@link Developer}
+ *
+ * Public classes that are not marked with this annotation must be
+ * considered by default as {@link Developer}.</li>
+ *
+ * External applications must only use classes that are marked {@link User}.
+ *
+ * Methods may have a different annotation that it is more restrictive
+ * compared to the audience classification of the class. Example: A class
+ * might be {@link User}, but a method may be {@link Developer}
+ */
+@InterfaceAudience.User
+@InterfaceStability.Evolving
+public class InterfaceAudience {
+  /**
+   * Intended for use by any project or application.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface User { }
+
+  /**
+   * Intended only for developers to extend interface for CarbonData project
+   * For example, new Datamap implementations.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Developer { }
+
+  private InterfaceAudience() { } // Audience can't exist on its own
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/97139299/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
new file mode 100644
index 0000000..b8e5e52
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.annotations;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience.*;
+
+/**
+ * This annotation is ported and modified from Apache Hadoop project.
+ *
+ * Annotation to inform users of how much to rely on a particular package,
+ * class or method not changing over time. Currently the stability can be
+ * {@link Stable}, {@link Evolving} or {@link Unstable}. <br>
+ *
+ * 1. {@link Stable} means compatibility can break only at major release (m.0)
+ * 2. {@link Evolving} means compatibility can break at minor release (m.x)
+ * 3. {@link Unstable} means compatibility can break at any release
+ *
+ * <ul><li>All classes that are annotated with {@link User} or
+ * {@link Developer} must have InterfaceStability annotation. </li>
+ * <li>Classes that are {@link Private} are to be considered unstable unless
+ * a different InterfaceStability annotation states otherwise.</li>
+ * <li>Incompatible changes must not be made to classes marked as stable.</li>
+ * </ul>
+ */
+@InterfaceAudience.User
+@org.apache.hadoop.classification.InterfaceStability.Evolving
+public class InterfaceStability {
+  /**
+   * Can evolve while retaining compatibility for minor release boundaries.;
+   * can break compatibility only at major release (ie. at m.0).
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Stable { }
+
+  /**
+   * Evolving, but can break compatibility at minor release (i.e. m.x)
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Evolving { }
+
+  /**
+   * No guarantee is provided as to reliability or stability across any
+   * level of release granularity.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Unstable { }
+}


[43/49] carbondata git commit: [CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

Posted by ja...@apache.org.
[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

Carbondata assign blocks to nodes at the beginning of data loading.
Previous block allocation strategy is block number based and it will
suffer skewed data problem if the size of input files differs a lot.

We introduced a size based block allocation strategy to optimize data
loading performance in skewed data scenario.

This closes #1808


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

Branch: refs/heads/carbonstore-rebase4
Commit: d6d6f10bc7e4e8ee5e8d1a9ee2b723ada37564a3
Parents: 93b2efd
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Thu Feb 8 14:42:39 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:20:37 2018 +0800

----------------------------------------------------------------------
 .../constants/CarbonLoadOptionConstants.java    |  10 +
 .../core/datastore/block/TableBlockInfo.java    |  29 ++
 .../carbondata/core/util/CarbonProperties.java  |  11 +
 docs/useful-tips-on-carbondata.md               |   1 +
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |   4 +-
 .../spark/sql/hive/DistributionUtil.scala       |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  18 +-
 .../merger/NodeMultiBlockRelation.java          |  40 ++
 .../processing/util/CarbonLoaderUtil.java       | 480 ++++++++++++-------
 .../processing/util/CarbonLoaderUtilTest.java   | 125 +++++
 10 files changed, 545 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index bcfeba0..a6bf60f 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,4 +114,14 @@ public final class CarbonLoadOptionConstants {
    */
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 10000000;
 
+  /**
+   * enable block size based block allocation while loading data. By default, carbondata assigns
+   * blocks to node based on block number. If this option is set to `true`, carbondata will
+   * consider block size first and make sure that all the nodes will process almost equal size of
+   * data. This option is especially useful when you encounter skewed data.
+   */
+  @CarbonProperty
+  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
+      = "carbon.load.skewedDataOptimization.enabled";
+  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index a7bfdba..c0cebe0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -98,6 +100,20 @@ public class TableBlockInfo implements Distributable, Serializable {
 
   private String dataMapWriterPath;
 
+  /**
+   * comparator to sort by block size in descending order.
+   * Since each line is not exactly the same, the size of a InputSplit may differs,
+   * so we allow some deviation for these splits.
+   */
+  public static final Comparator<Distributable> DATA_SIZE_DESC_COMPARATOR =
+      new Comparator<Distributable>() {
+        @Override public int compare(Distributable o1, Distributable o2) {
+          long diff =
+              ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) o2).getBlockLength();
+          return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
+        }
+      };
+
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
       String[] locations, long blockLength, ColumnarFormatVersion version,
       String[] deletedDeltaFilePath) {
@@ -434,4 +450,17 @@ public class TableBlockInfo implements Distributable, Serializable {
   public void setDataMapWriterPath(String dataMapWriterPath) {
     this.dataMapWriterPath = dataMapWriterPath;
   }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("TableBlockInfo{");
+    sb.append("filePath='").append(filePath).append('\'');
+    sb.append(", blockOffset=").append(blockOffset);
+    sb.append(", blockLength=").append(blockLength);
+    sb.append(", segmentId='").append(segmentId).append('\'');
+    sb.append(", blockletId='").append(blockletId).append('\'');
+    sb.append(", locations=").append(Arrays.toString(locations));
+    sb.append('}');
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index 9d52669..474988c 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -1235,6 +1235,17 @@ public final class CarbonProperties {
       return CarbonCommonConstants.CARBON_SORT_TEMP_COMPRESSOR_DEFAULT;
     }
   }
+
+  /**
+   * whether optimization for skewed data is enabled
+   * @return true, if enabled; false for not enabled.
+   */
+  public boolean isLoadSkewedDataOptimizationEnabled() {
+    String skewedEnabled = getProperty(
+        CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION,
+        CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT);
+    return skewedEnabled.equalsIgnoreCase("true");
+  }
   /**
    * returns true if carbon property
    * @param key

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/docs/useful-tips-on-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/useful-tips-on-carbondata.md b/docs/useful-tips-on-carbondata.md
index 4d43003..ff339d0 100644
--- a/docs/useful-tips-on-carbondata.md
+++ b/docs/useful-tips-on-carbondata.md
@@ -169,5 +169,6 @@
   | carbon.use.local.dir | spark/carbonlib/carbon.properties | Data loading | Whether use YARN local directories for multi-table load disk load balance | If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance. |
   | carbon.use.multiple.temp.dir | spark/carbonlib/carbon.properties | Data loading | Whether to use multiple YARN local directories during table data loading for disk load balance | After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading. |
   | carbon.sort.temp.compressor | spark/carbonlib/carbon.properties | Data loading | Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading. | The optional values are 'SNAPPY','GZIP','BZIP2','LZ4' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck. |
+  | carbon.load.skewedDataOptimization.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable size based block allocation strategy for data loading. | When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB. |
 
   Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 8fc80b7..97e3061 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -281,8 +281,10 @@ class NewCarbonDataLoadRDD[K, V](
         val format = new CSVInputFormat
 
         val split = theSplit.asInstanceOf[CarbonNodePartition]
+        val inputSize = split.blocksDetails.map(_.getBlockLength).sum * 0.1 * 10  / 1024 / 1024
         logInfo("Input split: " + split.serializableHadoopSplit)
-        logInfo("The Block Count in this node :" + split.nodeBlocksDetail.length)
+        logInfo("The block count in this node: " + split.nodeBlocksDetail.length)
+        logInfo(f"The input data size in this node: $inputSize%.2fMB")
         CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordHostBlockMap(
             split.serializableHadoopSplit, split.nodeBlocksDetail.length)
         carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
index 1958d61..a676dd8 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
@@ -128,7 +128,7 @@ object DistributionUtil {
    */
   def ensureExecutorsAndGetNodeList(blockList: Seq[Distributable],
       sparkContext: SparkContext): Seq[String] = {
-    val nodeMapping = CarbonLoaderUtil.getRequiredExecutors(blockList.asJava)
+    val nodeMapping = CarbonLoaderUtil.nodeBlockMapping(blockList.asJava)
     ensureExecutorsByNumberAndGetNodeList(nodeMapping, blockList, sparkContext)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 6f8155c..759a8cc 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -1026,10 +1026,16 @@ object CarbonDataRDDFactory {
     val startTime = System.currentTimeMillis
     val activeNodes = DistributionUtil
       .ensureExecutorsAndGetNodeList(blockList, sqlContext.sparkContext)
-    val nodeBlockMapping =
-      CarbonLoaderUtil
-        .nodeBlockMapping(blockList.toSeq.asJava, -1, activeNodes.toList.asJava).asScala
-        .toSeq
+    val skewedDataOptimization = CarbonProperties.getInstance()
+      .isLoadSkewedDataOptimizationEnabled()
+    val blockAssignStrategy = if (skewedDataOptimization) {
+      CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_SIZE_FIRST
+    } else {
+      CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_NUM_FIRST
+    }
+    LOGGER.info(s"Allocating block to nodes using strategy: $blockAssignStrategy")
+    val nodeBlockMapping = CarbonLoaderUtil.nodeBlockMapping(blockList.toSeq.asJava, -1,
+      activeNodes.toList.asJava, blockAssignStrategy).asScala.toSeq
     val timeElapsed: Long = System.currentTimeMillis - startTime
     LOGGER.info("Total Time taken in block allocation: " + timeElapsed)
     LOGGER.info(s"Total no of blocks: ${ blockList.length }, " +
@@ -1037,7 +1043,9 @@ object CarbonDataRDDFactory {
     var str = ""
     nodeBlockMapping.foreach { entry =>
       val tableBlock = entry._2
-      str = str + "#Node: " + entry._1 + " no.of.blocks: " + tableBlock.size()
+      val totalSize = tableBlock.asScala.map(_.asInstanceOf[TableBlockInfo].getBlockLength).sum
+      str = str + "#Node: " + entry._1 + ", no.of.blocks: " + tableBlock.size() +
+            f", totalsize.of.blocks: ${totalSize * 0.1 * 10 / 1024 /1024}%.2fMB"
       tableBlock.asScala.foreach(tableBlockInfo =>
         if (!tableBlockInfo.getLocations.exists(hostentry =>
           hostentry.equalsIgnoreCase(entry._1)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java b/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
index ec2ddaf..1bb5736 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
@@ -16,15 +16,41 @@
  */
 package org.apache.carbondata.processing.merger;
 
+import java.util.Comparator;
 import java.util.List;
 
 import org.apache.carbondata.core.datastore.block.Distributable;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 
 public class NodeMultiBlockRelation implements Comparable<NodeMultiBlockRelation> {
 
   private final List<Distributable> blocks;
   private final String node;
 
+  /**
+   * comparator to sort by data size in descending order. This is used to assign big blocks to
+   * bigger nodes first.
+   */
+  public static final Comparator<NodeMultiBlockRelation> DATA_SIZE_DESC_COMPARATOR =
+      new Comparator<NodeMultiBlockRelation>() {
+        @Override
+        public int compare(NodeMultiBlockRelation o1, NodeMultiBlockRelation o2) {
+          long diff = o1.getTotalSizeOfBlocks() - o2.getTotalSizeOfBlocks();
+          return diff > 0 ? -1 : (diff < 0 ? 1 : 0);
+        }
+      };
+  /**
+   * comparator to sort by data size in ascending order. This is used to assign left over blocks to
+   * smaller nodes first.
+   */
+  public static final Comparator<NodeMultiBlockRelation> DATA_SIZE_ASC_COMPARATOR =
+      new Comparator<NodeMultiBlockRelation>() {
+        @Override
+        public int compare(NodeMultiBlockRelation o1, NodeMultiBlockRelation o2) {
+          long diff = o1.getTotalSizeOfBlocks() - o2.getTotalSizeOfBlocks();
+          return diff > 0 ? 1 : (diff < 0 ? -1 : 0);
+        }
+      };
   public NodeMultiBlockRelation(String node, List<Distributable> blocks) {
     this.node = node;
     this.blocks = blocks;
@@ -39,6 +65,20 @@ public class NodeMultiBlockRelation implements Comparable<NodeMultiBlockRelation
     return node;
   }
 
+  /**
+   * get the total size of the blocks
+   * @return size in bytes
+   */
+  public long getTotalSizeOfBlocks() {
+    long totalSize = 0;
+    if (blocks.get(0) instanceof TableBlockInfo) {
+      for (Distributable block : blocks) {
+        totalSize += ((TableBlockInfo) block).getBlockLength();
+      }
+    }
+    return totalSize;
+  }
+
   @Override public int compareTo(NodeMultiBlockRelation obj) {
     return this.blocks.size() - obj.getBlocks().size();
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 6876355..c2e224d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -31,6 +31,7 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.Distributable;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -48,7 +49,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-import org.apache.carbondata.processing.merger.NodeBlockRelation;
 import org.apache.carbondata.processing.merger.NodeMultiBlockRelation;
 
 import static org.apache.carbondata.core.enums.EscapeSequences.*;
@@ -62,6 +62,23 @@ public final class CarbonLoaderUtil {
   private CarbonLoaderUtil() {
   }
 
+  /**
+   * strategy for assign blocks to nodes/executors
+   */
+  public enum BlockAssignmentStrategy {
+    BLOCK_NUM_FIRST("Assign blocks to node base on number of blocks"),
+    BLOCK_SIZE_FIRST("Assign blocks to node base on data size of blocks");
+    private String name;
+    BlockAssignmentStrategy(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public String toString() {
+      return this.getClass().getSimpleName() + ':' + this.name;
+    }
+  }
+
   public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
     String segmentPath = CarbonTablePath.getSegmentPath(
         loadModel.getTablePath(), currentLoad + "");
@@ -441,9 +458,9 @@ public final class CarbonLoaderUtil {
   public static Map<String, List<List<Distributable>>> nodeBlockTaskMapping(
       List<Distributable> blockInfos, int noOfNodesInput, int parallelism,
       List<String> activeNode) {
-
     Map<String, List<Distributable>> mapOfNodes =
-        CarbonLoaderUtil.nodeBlockMapping(blockInfos, noOfNodesInput, activeNode);
+        CarbonLoaderUtil.nodeBlockMapping(blockInfos, noOfNodesInput, activeNode,
+            BlockAssignmentStrategy.BLOCK_NUM_FIRST);
     int taskPerNode = parallelism / mapOfNodes.size();
     //assigning non zero value to noOfTasksPerNode
     int noOfTasksPerNode = taskPerNode == 0 ? 1 : taskPerNode;
@@ -459,86 +476,75 @@ public final class CarbonLoaderUtil {
    */
   public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos,
       int noOfNodesInput) {
-    return nodeBlockMapping(blockInfos, noOfNodesInput, null);
+    return nodeBlockMapping(blockInfos, noOfNodesInput, null,
+        BlockAssignmentStrategy.BLOCK_NUM_FIRST);
   }
 
   /**
-   * the method returns the number of required executors
+   * This method will divide the blocks among the nodes as per the data locality
    *
    * @param blockInfos
    * @return
    */
-  public static Map<String, List<Distributable>> getRequiredExecutors(
-      List<Distributable> blockInfos) {
-    List<NodeBlockRelation> flattenedList =
-        new ArrayList<NodeBlockRelation>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (Distributable blockInfo : blockInfos) {
-      try {
-        for (String eachNode : blockInfo.getLocations()) {
-          NodeBlockRelation nbr = new NodeBlockRelation(blockInfo, eachNode);
-          flattenedList.add(nbr);
-        }
-      } catch (IOException e) {
-        throw new RuntimeException("error getting location of block: " + blockInfo.toString(), e);
-      }
-    }
-    // sort the flattened data.
-    Collections.sort(flattenedList);
-    Map<String, List<Distributable>> nodeAndBlockMapping =
-        new LinkedHashMap<String, List<Distributable>>(
-            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    // from the flattened list create a mapping of node vs Data blocks.
-    createNodeVsBlockMapping(flattenedList, nodeAndBlockMapping);
-    return nodeAndBlockMapping;
+  public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos) {
+    // -1 if number of nodes has to be decided based on block location information
+    return nodeBlockMapping(blockInfos, -1);
   }
 
   /**
    * This method will divide the blocks among the nodes as per the data locality
    *
-   * @param blockInfos
+   * @param blockInfos blocks
    * @param noOfNodesInput -1 if number of nodes has to be decided
    *                       based on block location information
-   * @return
+   * @param blockAssignmentStrategy strategy used to assign blocks
+   * @return a map that maps node to blocks
    */
-  public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos,
-      int noOfNodesInput, List<String> activeNodes) {
-
-    Map<String, List<Distributable>> nodeBlocksMap =
-        new HashMap<String, List<Distributable>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    List<NodeBlockRelation> flattenedList =
-        new ArrayList<NodeBlockRelation>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    Set<Distributable> uniqueBlocks =
-        new HashSet<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    Set<String> nodes = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    createFlattenedListFromMap(blockInfos, flattenedList, uniqueBlocks, nodes);
+  public static Map<String, List<Distributable>> nodeBlockMapping(
+      List<Distributable> blockInfos, int noOfNodesInput, List<String> activeNodes,
+      BlockAssignmentStrategy blockAssignmentStrategy) {
+    ArrayList<NodeMultiBlockRelation> rtnNode2Blocks = new ArrayList<>();
+
+    Set<Distributable> uniqueBlocks = new HashSet<>(blockInfos);
+    ArrayList<NodeMultiBlockRelation> originNode2Blocks = createNode2BlocksMapping(blockInfos);
+    Set<String> nodes = new HashSet<>(originNode2Blocks.size());
+    for (NodeMultiBlockRelation relation : originNode2Blocks) {
+      nodes.add(relation.getNode());
+    }
 
     int noofNodes = (-1 == noOfNodesInput) ? nodes.size() : noOfNodesInput;
     if (null != activeNodes) {
       noofNodes = activeNodes.size();
     }
-    int blocksPerNode = blockInfos.size() / noofNodes;
-    blocksPerNode = blocksPerNode <= 0 ? 1 : blocksPerNode;
-
-    // sort the flattened data.
-    Collections.sort(flattenedList);
 
-    Map<String, List<Distributable>> nodeAndBlockMapping =
-        new LinkedHashMap<String, List<Distributable>>(
-            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    // from the flattened list create a mapping of node vs Data blocks.
-    createNodeVsBlockMapping(flattenedList, nodeAndBlockMapping);
+    // calculate the average expected size for each node
+    long sizePerNode = 0;
+    if (BlockAssignmentStrategy.BLOCK_NUM_FIRST == blockAssignmentStrategy) {
+      sizePerNode = blockInfos.size() / noofNodes;
+      sizePerNode = sizePerNode <= 0 ? 1 : sizePerNode;
+    } else if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
+      long totalFileSize = 0;
+      for (Distributable blockInfo : uniqueBlocks) {
+        totalFileSize += ((TableBlockInfo) blockInfo).getBlockLength();
+      }
+      sizePerNode = totalFileSize / noofNodes;
+    }
 
-    // so now we have a map of node vs blocks. allocate the block as per the order
-    createOutputMap(nodeBlocksMap, blocksPerNode, uniqueBlocks, nodeAndBlockMapping, activeNodes);
+    // assign blocks to each node
+    assignBlocksByDataLocality(rtnNode2Blocks, sizePerNode, uniqueBlocks, originNode2Blocks,
+        activeNodes, blockAssignmentStrategy);
 
     // if any blocks remain then assign them to nodes in round robin.
-    assignLeftOverBlocks(nodeBlocksMap, uniqueBlocks, blocksPerNode, activeNodes);
+    assignLeftOverBlocks(rtnNode2Blocks, uniqueBlocks, sizePerNode, activeNodes,
+        blockAssignmentStrategy);
 
-    return nodeBlocksMap;
+    // convert
+    Map<String, List<Distributable>> rtnNodeBlocksMap =
+        new HashMap<String, List<Distributable>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    for (NodeMultiBlockRelation relation : rtnNode2Blocks) {
+      rtnNodeBlocksMap.put(relation.getNode(), relation.getBlocks());
+    }
+    return rtnNodeBlocksMap;
   }
 
   /**
@@ -613,92 +619,207 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * If any left over data blocks are present then assign those to nodes in round robin way.
-   *
-   * @param outputMap
-   * @param uniqueBlocks
+   * If any left over data blocks are present then assign those to nodes in round robin way. This
+   * will not obey the data locality.
    */
-  private static void assignLeftOverBlocks(Map<String, List<Distributable>> outputMap,
-      Set<Distributable> uniqueBlocks, int noOfBlocksPerNode, List<String> activeNodes) {
+  private static void assignLeftOverBlocks(ArrayList<NodeMultiBlockRelation> outputMap,
+      Set<Distributable> leftOverBlocks, long expectedSizePerNode, List<String> activeNodes,
+      BlockAssignmentStrategy blockAssignmentStrategy) {
+    Map<String, Integer> node2Idx = new HashMap<>(outputMap.size());
+    for (int idx = 0; idx < outputMap.size(); idx++) {
+      node2Idx.put(outputMap.get(idx).getNode(), idx);
+    }
 
+    // iterate all the nodes and try to allocate blocks to the nodes
     if (activeNodes != null) {
       for (String activeNode : activeNodes) {
-        List<Distributable> blockLst = outputMap.get(activeNode);
-        if (null == blockLst) {
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("Second assignment iteration: assign for executor: " + activeNode);
+        }
+
+        Integer idx;
+        List<Distributable> blockLst;
+        if (node2Idx.containsKey(activeNode)) {
+          idx = node2Idx.get(activeNode);
+          blockLst = outputMap.get(idx).getBlocks();
+        } else {
+          idx = node2Idx.size();
           blockLst = new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
         }
-        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
-        if (blockLst.size() > 0) {
-          outputMap.put(activeNode, blockLst);
+        populateBlocks(leftOverBlocks, expectedSizePerNode, blockLst, blockAssignmentStrategy);
+
+        if (!node2Idx.containsKey(activeNode) && blockLst.size() > 0) {
+          outputMap.add(idx, new NodeMultiBlockRelation(activeNode, blockLst));
+          node2Idx.put(activeNode, idx);
         }
       }
     } else {
-      for (Map.Entry<String, List<Distributable>> entry : outputMap.entrySet()) {
-        List<Distributable> blockLst = entry.getValue();
-        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
+      for (NodeMultiBlockRelation entry : outputMap) {
+        List<Distributable> blockLst = entry.getBlocks();
+        populateBlocks(leftOverBlocks, expectedSizePerNode, blockLst, blockAssignmentStrategy);
       }
-
     }
 
-    for (Map.Entry<String, List<Distributable>> entry : outputMap.entrySet()) {
-      Iterator<Distributable> blocks = uniqueBlocks.iterator();
-      if (blocks.hasNext()) {
-        Distributable block = blocks.next();
-        List<Distributable> blockLst = entry.getValue();
-        blockLst.add(block);
-        blocks.remove();
-      }
+    // if there is still blocks left, allocate them in round robin manner to each nodes
+    assignBlocksUseRoundRobin(outputMap, leftOverBlocks, blockAssignmentStrategy);
+  }
+
+  /**
+   * assign remaining blocks to nodes
+   *
+   * @param remainingBlocks blocks to be allocated
+   * @param expectedSizePerNode expected size for each node
+   * @param blockLst destination for the blocks to be allocated
+   * @param blockAssignmentStrategy block assignment stretegy
+   */
+  private static void populateBlocks(Set<Distributable> remainingBlocks,
+      long expectedSizePerNode, List<Distributable> blockLst,
+      BlockAssignmentStrategy blockAssignmentStrategy) {
+    switch (blockAssignmentStrategy) {
+      case BLOCK_NUM_FIRST:
+        populateBlocksByNum(remainingBlocks, expectedSizePerNode, blockLst);
+        break;
+      case BLOCK_SIZE_FIRST:
+        populateBlocksBySize(remainingBlocks, expectedSizePerNode, blockLst);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            "Unsupported block assignment strategy: " + blockAssignmentStrategy);
     }
   }
 
   /**
-   * The method populate the blockLst to be allocate to a specific node.
-   * @param uniqueBlocks
-   * @param noOfBlocksPerNode
-   * @param blockLst
+   * Taken N number of distributable blocks from {@param remainingBlocks} and add them to output
+   * {@param blockLst}. After added, the total number of {@param blockLst} is less
+   * than {@param expectedSizePerNode}.
    */
-  private static void populateBlocks(Set<Distributable> uniqueBlocks, int noOfBlocksPerNode,
-      List<Distributable> blockLst) {
-    Iterator<Distributable> blocks = uniqueBlocks.iterator();
-    //if the node is already having the per block nodes then avoid assign the extra blocks
-    if (blockLst.size() == noOfBlocksPerNode) {
+  private static void populateBlocksByNum(Set<Distributable> remainingBlocks,
+      long expectedSizePerNode, List<Distributable> blockLst) {
+    Iterator<Distributable> blocks = remainingBlocks.iterator();
+    // if the node is already having the per block nodes then avoid assign the extra blocks
+    if (blockLst.size() == expectedSizePerNode) {
       return;
     }
     while (blocks.hasNext()) {
       Distributable block = blocks.next();
       blockLst.add(block);
       blocks.remove();
-      if (blockLst.size() >= noOfBlocksPerNode) {
+      if (blockLst.size() >= expectedSizePerNode) {
         break;
       }
     }
   }
 
   /**
-   * To create the final output of the Node and Data blocks
-   *
-   * @param outputMap
-   * @param blocksPerNode
-   * @param uniqueBlocks
-   * @param nodeAndBlockMapping
-   * @param activeNodes
+   * Taken N number of distributable blocks from {@param remainingBlocks} and add them to output
+   * {@param blockLst}. After added, the total accumulated block size of {@param blockLst}
+   * is less than {@param expectedSizePerNode}.
    */
-  private static void createOutputMap(Map<String, List<Distributable>> outputMap, int blocksPerNode,
-      Set<Distributable> uniqueBlocks, Map<String, List<Distributable>> nodeAndBlockMapping,
-      List<String> activeNodes) {
+  private static void populateBlocksBySize(Set<Distributable> remainingBlocks,
+      long expectedSizePerNode, List<Distributable> blockLst) {
+    Iterator<Distributable> blocks = remainingBlocks.iterator();
+    //if the node is already having the avg node size then avoid assign the extra blocks
+    long fileSize = 0;
+    for (Distributable block : blockLst) {
+      fileSize += ((TableBlockInfo) block).getBlockLength();
+    }
+    if (fileSize >= expectedSizePerNode) {
+      LOGGER.debug("Capacity is full, skip allocate blocks on this node");
+      return;
+    }
 
-    ArrayList<NodeMultiBlockRelation> multiBlockRelations =
-        new ArrayList<>(nodeAndBlockMapping.size());
-    for (Map.Entry<String, List<Distributable>> entry : nodeAndBlockMapping.entrySet()) {
-      multiBlockRelations.add(new NodeMultiBlockRelation(entry.getKey(), entry.getValue()));
+    while (blocks.hasNext()) {
+      Distributable block = blocks.next();
+      long thisBlockSize = ((TableBlockInfo) block).getBlockLength();
+      if (fileSize < expectedSizePerNode) {
+        // `fileSize==0` means there are no blocks assigned to this node before
+        if (fileSize == 0 || fileSize + thisBlockSize <= expectedSizePerNode * 1.1D) {
+          blockLst.add(block);
+          if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Second Assignment iteration: "
+                + ((TableBlockInfo) block).getFilePath() + "-"
+                + ((TableBlockInfo) block).getBlockLength() + "-->currentNode");
+          }
+          fileSize += thisBlockSize;
+          blocks.remove();
+        }
+      } else {
+        break;
+      }
+    }
+  }
+
+  /**
+   * allocate the blocks in round robin manner
+   */
+  private static void assignBlocksUseRoundRobin(ArrayList<NodeMultiBlockRelation> node2Blocks,
+      Set<Distributable> remainingBlocks, BlockAssignmentStrategy blockAssignmentStrategy) {
+    switch (blockAssignmentStrategy) {
+      case BLOCK_NUM_FIRST:
+        roundRobinAssignBlocksByNum(node2Blocks, remainingBlocks);
+        break;
+      case BLOCK_SIZE_FIRST:
+        roundRobinAssignBlocksBySize(node2Blocks, remainingBlocks);
+        break;
+      default:
+        throw new IllegalArgumentException("Unsupported block assignment strategy: "
+            + blockAssignmentStrategy);
+    }
+  }
+
+  private static void roundRobinAssignBlocksByNum(ArrayList<NodeMultiBlockRelation> outputMap,
+      Set<Distributable> remainingBlocks) {
+    for (NodeMultiBlockRelation relation: outputMap) {
+      Iterator<Distributable> blocks = remainingBlocks.iterator();
+      if (blocks.hasNext()) {
+        Distributable block = blocks.next();
+        List<Distributable> blockLst = relation.getBlocks();
+        blockLst.add(block);
+        blocks.remove();
+      }
+    }
+  }
+
+  private static void roundRobinAssignBlocksBySize(ArrayList<NodeMultiBlockRelation> outputMap,
+      Set<Distributable> remainingBlocks) {
+    Iterator<Distributable> blocks = remainingBlocks.iterator();
+    while (blocks.hasNext()) {
+      // sort the allocated node-2-blocks in ascending order, the total data size of first one is
+      // the smallest, so we assign this block to it.
+      Collections.sort(outputMap, NodeMultiBlockRelation.DATA_SIZE_ASC_COMPARATOR);
+      Distributable block = blocks.next();
+      List<Distributable> blockLst = outputMap.get(0).getBlocks();
+      blockLst.add(block);
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("RoundRobin assignment iteration: "
+            + ((TableBlockInfo) block).getFilePath() + "-"
+            + ((TableBlockInfo) block).getBlockLength() + "-->" + outputMap.get(0).getNode());
+      }
+      blocks.remove();
+    }
+  }
+  /**
+   * allocate distributable blocks to nodes based on data locality
+   */
+  private static void assignBlocksByDataLocality(
+      ArrayList<NodeMultiBlockRelation> outputNode2Blocks,
+      long expectedSizePerNode, Set<Distributable> remainingBlocks,
+      List<NodeMultiBlockRelation> inputNode2Blocks, List<String> activeNodes,
+      BlockAssignmentStrategy blockAssignmentStrategy) {
+    if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
+      // sort nodes based on data size of all blocks per node, so that nodes having bigger size
+      // are assigned first
+      Collections.sort(inputNode2Blocks, NodeMultiBlockRelation.DATA_SIZE_DESC_COMPARATOR);
+    } else {
+      // sort nodes based on number of blocks per node, so that nodes having lesser blocks
+      // are assigned first
+      Collections.sort(inputNode2Blocks);
     }
-    // sort nodes based on number of blocks per node, so that nodes having lesser blocks
-    // are assigned first
-    Collections.sort(multiBlockRelations);
 
-    for (NodeMultiBlockRelation nodeMultiBlockRelation : multiBlockRelations) {
+    Map<String, Integer> executor2Idx = new HashMap<>();
+    for (NodeMultiBlockRelation nodeMultiBlockRelation : inputNode2Blocks) {
       String nodeName = nodeMultiBlockRelation.getNode();
-      //assign the block to the node only if the node is active
+      // assign the block to the node only if the node is active
       String activeExecutor = nodeName;
       if (null != activeNodes) {
         activeExecutor = getActiveExecutor(activeNodes, nodeName);
@@ -706,29 +827,75 @@ public final class CarbonLoaderUtil {
           continue;
         }
       }
-      // this loop will be for each NODE
-      int nodeCapacity = 0;
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("First Assignment iteration: assign for executor: " + activeExecutor);
+      }
+
+      List<Distributable> blocksInThisNode = nodeMultiBlockRelation.getBlocks();
+      if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
+        // sort blocks based on block size, so that bigger blocks will be assigned first
+        Collections.sort(blocksInThisNode, TableBlockInfo.DATA_SIZE_DESC_COMPARATOR);
+      }
+
+      long nodeCapacity = 0;
       // loop thru blocks of each Node
       for (Distributable block : nodeMultiBlockRelation.getBlocks()) {
+        if (!remainingBlocks.contains(block)) {
+          // this block has been added before
+          continue;
+        }
+        // this is the first time to add block to this node, initialize it
+        if (!executor2Idx.containsKey(activeExecutor)) {
+          Integer idx = executor2Idx.size();
+          outputNode2Blocks.add(idx, new NodeMultiBlockRelation(activeExecutor,
+              new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE)));
+          executor2Idx.put(activeExecutor, idx);
+        }
 
-        // check if this is already assigned.
-        if (uniqueBlocks.contains(block)) {
-
-          if (null == outputMap.get(activeExecutor)) {
-            List<Distributable> list =
-                new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-            outputMap.put(activeExecutor, list);
-          }
-          // assign this block to this node if node has capacity left
-          if (nodeCapacity < blocksPerNode) {
-            List<Distributable> infos = outputMap.get(activeExecutor);
+        // assign this block to this node if node has capacity left
+        if (BlockAssignmentStrategy.BLOCK_NUM_FIRST == blockAssignmentStrategy) {
+          if (nodeCapacity < expectedSizePerNode) {
+            Integer idx = executor2Idx.get(activeExecutor);
+            List<Distributable> infos = outputNode2Blocks.get(idx).getBlocks();
             infos.add(block);
             nodeCapacity++;
-            uniqueBlocks.remove(block);
+            if (LOGGER.isDebugEnabled()) {
+              LOGGER.debug(
+                  "First Assignment iteration: " + ((TableBlockInfo) block).getFilePath() + '-'
+                      + ((TableBlockInfo) block).getBlockLength() + "-->" + activeExecutor);
+            }
+            remainingBlocks.remove(block);
+          } else {
+            // No need to continue loop as node is full
+            break;
+          }
+        } else if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
+          long thisBlockSize = ((TableBlockInfo) block).getBlockLength();
+          // `nodeCapacity == 0` means that there is a huge block that already exceed the
+          // `expectedSize` of the node, so we have to assign it to some node, otherwise it will
+          // be assigned in the last RoundRobin iteration.
+          if (nodeCapacity == 0 || nodeCapacity < expectedSizePerNode) {
+            if (nodeCapacity == 0 || nodeCapacity + thisBlockSize <= expectedSizePerNode * 1.05D) {
+              Integer idx = executor2Idx.get(activeExecutor);
+              List<Distributable> blocks = outputNode2Blocks.get(idx).getBlocks();
+              blocks.add(block);
+              nodeCapacity += thisBlockSize;
+              if (LOGGER.isDebugEnabled()) {
+                LOGGER.debug(
+                    "First Assignment iteration: " + ((TableBlockInfo) block).getFilePath() + '-'
+                        + ((TableBlockInfo) block).getBlockLength() + "-->" + activeExecutor);
+              }
+              remainingBlocks.remove(block);
+            }
+            // this block is too big for current node and there are still capacity left
+            // for small files, so continue to allocate block on this node in next iteration.
           } else {
             // No need to continue loop as node is full
             break;
           }
+        } else {
+          throw new IllegalArgumentException(
+              "Unsupported block assignment strategy: " + blockAssignmentStrategy);
         }
       }
     }
@@ -772,60 +939,37 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * Create the Node and its related blocks Mapping and put in a Map
+   * Create node to blocks mapping
    *
-   * @param flattenedList
-   * @param nodeAndBlockMapping
+   * @param blockInfos input block info
    */
-  private static void createNodeVsBlockMapping(List<NodeBlockRelation> flattenedList,
-      Map<String, List<Distributable>> nodeAndBlockMapping) {
-    for (NodeBlockRelation nbr : flattenedList) {
-      String node = nbr.getNode();
-      List<Distributable> list;
-
-      if (null == nodeAndBlockMapping.get(node)) {
-        list = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        list.add(nbr.getBlock());
-        nodeAndBlockMapping.put(node, list);
-      } else {
-        list = nodeAndBlockMapping.get(node);
-        list.add(nbr.getBlock());
-      }
-    }
-    /*for resolving performance issue, removed values() with entrySet () iterating the values and
-    sorting it.entrySet will give the logical view for hashMap and we dont query the map twice for
-    each key whereas values () iterate twice*/
-    Iterator<Map.Entry<String, List<Distributable>>> iterator =
-        nodeAndBlockMapping.entrySet().iterator();
-    while (iterator.hasNext()) {
-      Collections.sort(iterator.next().getValue());
-    }
-  }
+  private static ArrayList<NodeMultiBlockRelation> createNode2BlocksMapping(
+      List<Distributable> blockInfos) {
+    Map<String, Integer> node2Idx = new HashMap<>();
+    ArrayList<NodeMultiBlockRelation> node2Blocks = new ArrayList<>();
 
-  /**
-   * Create the flat List i.e flattening of the Map.
-   *
-   * @param blockInfos
-   * @param flattenedList
-   * @param uniqueBlocks
-   */
-  private static void createFlattenedListFromMap(List<Distributable> blockInfos,
-      List<NodeBlockRelation> flattenedList, Set<Distributable> uniqueBlocks,
-      Set<String> nodeList) {
     for (Distributable blockInfo : blockInfos) {
-      // put the blocks in the set
-      uniqueBlocks.add(blockInfo);
-
       try {
-        for (String eachNode : blockInfo.getLocations()) {
-          NodeBlockRelation nbr = new NodeBlockRelation(blockInfo, eachNode);
-          flattenedList.add(nbr);
-          nodeList.add(eachNode);
+        for (final String eachNode : blockInfo.getLocations()) {
+          if (node2Idx.containsKey(eachNode)) {
+            Integer idx = node2Idx.get(eachNode);
+            List<Distributable> blocks = node2Blocks.get(idx).getBlocks();
+            blocks.add(blockInfo);
+          } else {
+            // add blocks to this node for the first time
+            Integer idx = node2Idx.size();
+            List<Distributable> blocks = new ArrayList<>();
+            blocks.add(blockInfo);
+            node2Blocks.add(idx, new NodeMultiBlockRelation(eachNode, blocks));
+            node2Idx.put(eachNode, idx);
+          }
         }
       } catch (IOException e) {
         throw new RuntimeException("error getting location of block: " + blockInfo.toString(), e);
       }
     }
+
+    return node2Blocks;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/d6d6f10b/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java b/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
new file mode 100644
index 0000000..9c66ada
--- /dev/null
+++ b/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.block.Distributable;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CarbonLoaderUtilTest {
+  private final static LogService LOGGER
+      = LogServiceFactory.getLogService(CarbonLoaderUtilTest.class.getName());
+
+  private List<Distributable> generateBlocks() {
+    List<Distributable> blockInfos = new ArrayList<>();
+    String filePath = "/fakepath";
+    String blockId = "1";
+
+    String[] locations = new String[] { "host2", "host3" };
+    ColumnarFormatVersion version = ColumnarFormatVersion.V1;
+
+    TableBlockInfo tableBlockInfo1 = new TableBlockInfo(filePath + "_a", 0,
+        blockId, locations, 30 * 1024 * 1024, version, null);
+    blockInfos.add(tableBlockInfo1);
+
+    TableBlockInfo tableBlockInfo2 = new TableBlockInfo(filePath + "_b", 0,
+        blockId, locations, 40 * 1024 * 1024, version, null);
+    blockInfos.add(tableBlockInfo2);
+
+    TableBlockInfo tableBlockInfo3 = new TableBlockInfo(filePath + "_c", 0,
+        blockId, locations, 20 * 1024 * 1024, version, null);
+    blockInfos.add(tableBlockInfo3);
+
+    TableBlockInfo tableBlockInfo4 = new TableBlockInfo(filePath + "_d", 0,
+        blockId, locations, 1, version, null);
+    blockInfos.add(tableBlockInfo4);
+
+    TableBlockInfo tableBlockInfo5 = new TableBlockInfo(filePath + "_e", 0,
+        blockId, locations, 1, version, null);
+    blockInfos.add(tableBlockInfo5);
+
+    TableBlockInfo tableBlockInfo6 = new TableBlockInfo(filePath + "_f", 0,
+        blockId, locations, 1, version, null);
+    blockInfos.add(tableBlockInfo6);
+
+    TableBlockInfo tableBlockInfo7 = new TableBlockInfo(filePath + "_g", 0,
+        blockId, locations, 1, version, null);
+    blockInfos.add(tableBlockInfo7);
+    return blockInfos;
+  }
+
+  private List<String> generateExecutors() {
+    List<String> activeNodes = new ArrayList<>();
+    activeNodes.add("host1");
+    activeNodes.add("host2");
+    activeNodes.add("host3");
+    return activeNodes;
+  }
+
+  @Test
+  public void testNodeBlockMappingByDataSize() throws Exception {
+    List<Distributable> blockInfos = generateBlocks();
+    List<String> activeNodes = generateExecutors();
+
+    // the blocks are assigned by size, so the number of block for each node are different
+    Map<String, List<Distributable>> nodeMappingBySize =
+        CarbonLoaderUtil.nodeBlockMapping(blockInfos, -1, activeNodes,
+            CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_SIZE_FIRST);
+    LOGGER.info(convertMapListAsString(nodeMappingBySize));
+    Assert.assertEquals(3, nodeMappingBySize.size());
+    for (Map.Entry<String, List<Distributable>> entry : nodeMappingBySize.entrySet()) {
+      if (entry.getValue().size() == 1) {
+        // only contains the biggest block
+        Assert.assertEquals(40 * 1024 * 1024L,
+            ((TableBlockInfo) entry.getValue().get(0)).getBlockLength());
+      } else {
+        Assert.assertTrue(entry.getValue().size() > 1);
+      }
+    }
+
+    // the blocks are assigned by number, so the number of blocks for each node are nearly the same
+    Map<String, List<Distributable>> nodeMappingByNum =
+        CarbonLoaderUtil.nodeBlockMapping(blockInfos, -1, activeNodes,
+            CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_NUM_FIRST);
+    LOGGER.info(convertMapListAsString(nodeMappingByNum));
+    Assert.assertEquals(3, nodeMappingBySize.size());
+    for (Map.Entry<String, List<Distributable>> entry : nodeMappingByNum.entrySet()) {
+      Assert.assertTrue(entry.getValue().size() == blockInfos.size() / 3
+          || entry.getValue().size() == blockInfos.size() / 3 + 1);
+    }
+  }
+
+  private <K, T> String convertMapListAsString(Map<K, List<T>> mapList) {
+    StringBuffer sb = new StringBuffer();
+    for (Map.Entry<K, List<T>> entry : mapList.entrySet()) {
+      String key = entry.getKey().toString();
+      String value = StringUtils.join(entry.getValue(), ", ");
+      sb.append(key).append(" -- ").append(value).append(System.lineSeparator());
+    }
+    return sb.toString();
+  }
+}
\ No newline at end of file


[49/49] carbondata git commit: [REBASE] Solve conflict after merging master

Posted by ja...@apache.org.
[REBASE] Solve conflict after merging master


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

Branch: refs/heads/carbonstore-rebase4
Commit: 7466d65382eff27f86a3701e20bbea5019bb9d43
Parents: de36a5d
Author: Jacky Li <ja...@qq.com>
Authored: Tue Feb 27 11:26:30 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 11:26:30 2018 +0800

----------------------------------------------------------------------
 .../ConcurrentOperationException.java           | 50 --------------------
 .../carbondata/core/datamap/TableDataMap.java   |  5 +-
 .../carbondata/core/datamap/dev/DataMap.java    |  9 +---
 .../core/datamap/dev/DataMapFactory.java        |  2 +-
 .../core/indexstore/BlockletDetailsFetcher.java |  3 +-
 .../blockletindex/BlockletDataMap.java          |  3 +-
 .../blockletindex/SegmentIndexFileStore.java    |  2 -
 .../core/metadata/PartitionMapFileStore.java    |  0
 .../statusmanager/SegmentStatusManager.java     | 10 +++-
 .../SegmentUpdateStatusManager.java             |  7 +--
 .../hadoop/api/CarbonTableInputFormat.java      |  5 +-
 .../preaggregate/TestPreAggCreateCommand.scala  |  2 +-
 .../TestInsertAndOtherCommandConcurrent.scala   |  2 +-
 .../StandardPartitionGlobalSortTestCase.scala   |  2 +-
 .../exception/ProcessMetaDataException.java     |  2 +
 .../org/apache/carbondata/api/CarbonStore.scala |  6 +--
 .../load/DataLoadProcessBuilderOnSpark.scala    |  4 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |  2 +-
 .../carbondata/spark/util/CommonUtil.scala      |  2 -
 .../command/carbonTableSchemaCommon.scala       |  6 +--
 .../CarbonAlterTableCompactionCommand.scala     | 14 +-----
 .../management/CarbonCleanFilesCommand.scala    |  2 +-
 .../CarbonDeleteLoadByIdCommand.scala           |  2 +-
 .../CarbonDeleteLoadByLoadDateCommand.scala     |  2 +-
 .../management/CarbonLoadDataCommand.scala      | 28 +++++------
 .../CarbonProjectForDeleteCommand.scala         |  2 +-
 .../CarbonProjectForUpdateCommand.scala         |  2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |  2 +-
 .../command/table/CarbonDropTableCommand.scala  |  2 +-
 .../datasources/CarbonFileFormat.scala          |  3 --
 .../vectorreader/AddColumnTestCases.scala       |  1 +
 .../datamap/DataMapWriterListener.java          |  3 +-
 .../loading/model/CarbonLoadModelBuilder.java   | 34 ++++++++++++-
 .../processing/loading/model/LoadOption.java    | 15 +++++-
 .../processing/merger/CarbonDataMergerUtil.java | 19 +++++---
 .../merger/CompactionResultSortProcessor.java   |  4 +-
 .../merger/RowResultMergerProcessor.java        |  4 +-
 .../partition/spliter/RowResultProcessor.java   |  4 +-
 .../util/CarbonDataProcessorUtil.java           |  3 +-
 store/sdk/pom.xml                               |  2 +-
 40 files changed, 130 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/common/src/main/java/org/apache/carbondata/common/exceptions/ConcurrentOperationException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/ConcurrentOperationException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/ConcurrentOperationException.java
deleted file mode 100644
index 7308100..0000000
--- a/common/src/main/java/org/apache/carbondata/common/exceptions/ConcurrentOperationException.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.common.exceptions;
-
-import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.common.annotations.InterfaceStability;
-
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-
-/**
- * This exception will be thrown when executing concurrent operations which
- * is not supported in carbon.
- *
- * For example, when INSERT OVERWRITE is executing, other operations are not
- * allowed, so this exception will be thrown
- */
-@InterfaceAudience.User
-@InterfaceStability.Stable
-public class ConcurrentOperationException extends Exception {
-
-  public ConcurrentOperationException(String dbName, String tableName, String command1,
-      String command2) {
-    super(command1 + " is in progress for table " + dbName + "." + tableName + ", " + command2 +
-      " operation is not allowed");
-  }
-
-  public ConcurrentOperationException(CarbonTable table, String command1, String command2) {
-    this(table.getDatabaseName(), table.getTableName(), command1, command2);
-  }
-
-  public String getMessage() {
-    return super.getMessage();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index 2a6ceaa..5a01ec1 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -29,8 +29,8 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -160,7 +160,8 @@ public final class TableDataMap extends OperationEventListener {
             writePath + CarbonCommonConstants.FILE_SEPARATOR + System.nanoTime();
         detailedBlocklet.setDataMapWriterPath(blockletwritePath);
         serializer.serializeBlocklet((FineGrainBlocklet) blocklet, blockletwritePath);
-      }detailedBlocklet.setSegmentId(distributable.getSegment().getSegmentNo());
+      }
+      detailedBlocklet.setSegmentId(distributable.getSegment().getSegmentNo());
       detailedBlocklets.add(detailedBlocklet);
     }
     return detailedBlocklets;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
index 4a68286..fdeacff 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -38,18 +38,13 @@ public interface DataMap<T extends Blocklet> {
   /**
    * Prune the datamap with filter expression and partition information. It returns the list of
    * blocklets where these filters can exist.
-   *
-   * @param filterExp
-   * @return
    */
-  List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties, List<PartitionSpec> partitions);
+  List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
+      List<PartitionSpec> partitions);
 
   // TODO Move this method to Abstract class
   /**
    * Validate whether the current segment needs to be fetching the required data
-   *
-   * @param filterExp
-   * @return
    */
   boolean isScanRequired(FilterResolverIntf filterExp);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index 50ac279..d8a467f 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -21,8 +21,8 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.events.Event;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
index dd592c0..58c11db 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
@@ -53,5 +53,6 @@ public interface BlockletDetailsFetcher {
    * @param segment
    * @return
    */
-  List<Blocklet> getAllBlocklets(Segment segment, List<PartitionSpec> partitions) throws IOException;
+  List<Blocklet> getAllBlocklets(Segment segment, List<PartitionSpec> partitions)
+      throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index ce6193b..b379ae3 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -660,7 +660,8 @@ public class BlockletDataMap extends AbstractCoarseGrainDataMap implements Cache
   }
 
   @Override
-  public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties, List<PartitionSpec> partitions) {
+  public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties,
+      List<PartitionSpec> partitions) {
     if (unsafeMemoryDMStore.getRowCount() == 0) {
       return new ArrayList<>();
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
index 537e124..00d03a5 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/SegmentIndexFileStore.java
@@ -21,10 +21,8 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
deleted file mode 100644
index e69de29..0000000

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index 89666ab..1bb4b03 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -39,6 +39,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
 import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
 import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.locks.CarbonLockFactory;
 import org.apache.carbondata.core.locks.CarbonLockUtil;
 import org.apache.carbondata.core.locks.ICarbonLock;
@@ -838,6 +839,13 @@ public class SegmentStatusManager {
   public static void deleteLoadsAndUpdateMetadata(
       CarbonTable carbonTable,
       boolean isForceDeletion) throws IOException {
+    deleteLoadsAndUpdateMetadata(carbonTable, isForceDeletion, null);
+  }
+
+  public static void deleteLoadsAndUpdateMetadata(
+      CarbonTable carbonTable,
+      boolean isForceDeletion,
+      List<PartitionSpec> partitionSpecs) throws IOException {
     if (isLoadDeletionRequired(carbonTable.getMetadataPath())) {
       LoadMetadataDetails[] details =
           SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
@@ -882,7 +890,7 @@ public class SegmentStatusManager {
           CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.TABLE_STATUS_LOCK);
           if (updationCompletionStatus) {
             DeleteLoadFolders.physicalFactAndMeasureMetadataDeletion(
-                identifier, carbonTable.getMetadataPath(), isForceDeletion);
+                identifier, carbonTable.getMetadataPath(), isForceDeletion, partitionSpecs);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
index 6ec6fa2..4a2149e 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
@@ -77,8 +77,8 @@ public class SegmentUpdateStatusManager {
     this.identifier = identifier;
     // current it is used only for read function scenarios, as file update always requires to work
     // on latest file status.
-    segmentDetails =
-        SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(identifier.getTablePath()));
+    segmentDetails = SegmentStatusManager.readLoadMetadata(
+        CarbonTablePath.getMetadataPath(identifier.getTablePath()));
     if (segmentDetails.length > 0) {
       isPartitionTable = segmentDetails[0].getSegmentFile() != null;
     }
@@ -259,7 +259,8 @@ public class SegmentUpdateStatusManager {
             + CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.PART_ID)
             .replace("#", "/") + CarbonCommonConstants.FILE_SEPARATOR + completeBlockName;
       } else {
-        String carbonDataDirectoryPath = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment);
+        String carbonDataDirectoryPath =
+            CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment);
         blockPath =
             carbonDataDirectoryPath + CarbonCommonConstants.FILE_SEPARATOR + completeBlockName;
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index b485b69..5184e07 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -33,8 +33,8 @@ import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
-import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -494,7 +494,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
       long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
       long maxSize = getMaxSplitSize(job);
       for (Segment segment : streamSegments) {
-        String segmentDir = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment.getSegmentNo());
+        String segmentDir = CarbonTablePath.getSegmentPath(
+            identifier.getTablePath(), segment.getSegmentNo());
         FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
         if (FileFactory.isFileExist(segmentDir, fileType)) {
           String indexName = CarbonTablePath.getCarbonStreamIndexFileName();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
index 8f63af6..e09b922 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
index b39c44c..3f0ca42 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
@@ -34,11 +34,11 @@ import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainData
 import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMap}
 import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
 import org.apache.carbondata.core.datastore.page.ColumnPage
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.events.Event
-import org.apache.carbondata.spark.exception.ConcurrentOperationException
 import org.apache.carbondata.spark.testsuite.datamap.C2DataMapFactory
 
 // This testsuite test insert and insert overwrite with other commands concurrently

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
index 7d0959c..629417f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionGlobalSortTestCase.scala
@@ -24,9 +24,9 @@ import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.common.constants.LoggerAction
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 class StandardPartitionGlobalSortTestCase extends QueryTest with BeforeAndAfterAll {
   var executorService: ExecutorService = _

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ProcessMetaDataException.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ProcessMetaDataException.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ProcessMetaDataException.java
index 3e06bde..471b645 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ProcessMetaDataException.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ProcessMetaDataException.java
@@ -17,6 +17,8 @@
 
 package org.apache.carbondata.spark.exception;
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException;
+
 // This exception will be thrown when processMetaData failed in
 // Carbon's RunnableCommand
 public class ProcessMetaDataException extends MalformedCarbonCommandException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
index b69ec37..bfb1616 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
@@ -36,7 +36,6 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFile
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.spark.util.DataLoadingUtil
 
 object CarbonStore {
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
@@ -139,9 +138,8 @@ object CarbonStore {
         carbonCleanFilesLock =
           CarbonLockUtil
             .getLockObject(absoluteTableIdentifier, LockUsage.CLEAN_FILES_LOCK, errorMsg)
-        SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable, true)
-        DataLoadingUtil.deleteLoadsAndUpdateMetadata(
-          isForceDeletion = true, carbonTable, currentTablePartitions.map(_.asJava).orNull)
+        SegmentStatusManager.deleteLoadsAndUpdateMetadata(
+          carbonTable, true, currentTablePartitions.map(_.asJava).orNull)
         CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, true)
         currentTablePartitions match {
           case Some(partitions) =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
index e1bd84b..2648642 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
@@ -24,6 +24,7 @@ import org.apache.spark.TaskContext
 import org.apache.spark.sql.{DataFrame, SparkSession}
 import org.apache.spark.sql.execution.command.ExecutionErrors
 import org.apache.spark.storage.StorageLevel
+import org.apache.spark.util.CsvRDDHelper
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -34,7 +35,6 @@ import org.apache.carbondata.processing.loading.{DataLoadProcessBuilder, Failure
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.sort.sortdata.{NewRowComparator, NewRowComparatorForNormalDims, SortParameters}
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
-import org.apache.carbondata.spark.util.DataLoadingUtil
 
 /**
  * Use sortBy operator in spark to load the data
@@ -52,7 +52,7 @@ object DataLoadProcessBuilderOnSpark {
     } else {
       // input data from files
       val columnCount = model.getCsvHeaderColumns.length
-      DataLoadingUtil.csvFileScanRDD(sparkSession, model, hadoopConf)
+      CsvRDDHelper.csvFileScanRDD(sparkSession, model, hadoopConf)
         .map(DataLoadProcessorStepOnSpark.toStringArrayRow(_, columnCount))
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index 33263d6..298c84e 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.execution.command.{DataTypeInfo, UpdateTableModel}
 import org.apache.spark.sql.types._
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogService
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
@@ -45,7 +46,6 @@ import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 object CarbonScalaUtil {
   def convertSparkToCarbonDataType(dataType: DataType): CarbonDataType = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 9104a32..d3093fb 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -816,8 +816,6 @@ object CommonUtil {
                   val carbonTable = CarbonMetadata.getInstance
                     .getCarbonTable(identifier.getCarbonTableIdentifier.getTableUniqueName)
                   SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable, true)
-                  DataLoadingUtil.deleteLoadsAndUpdateMetadata(
-                    isForceDeletion = true, carbonTable, null)
                 } catch {
                   case _: Exception =>
                     LOGGER.warn(s"Error while cleaning table " +

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 71ce2c6..3c21af3 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -31,19 +31,15 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.Segment
 import org.apache.carbondata.core.indexstore.PartitionSpec
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier
-import org.apache.carbondata.core.metadata.SegmentFileStore.SegmentFile
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.datatype.{DataType, DataTypes, DecimalType}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema._
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier, TableInfo, TableSchema}
 import org.apache.carbondata.core.metadata.schema.table.column.{ColumnSchema, ParentColumnTableRelation}
-import org.apache.carbondata.core.service.CarbonCommonFactory
 import org.apache.carbondata.core.service.impl.ColumnUniqueIdGenerator
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentUpdateStatusManager}
 import org.apache.carbondata.core.util.DataTypeUtil
-import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.CompactionType

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 75de1fe..cdea4c8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -21,7 +21,7 @@ import java.io.{File, IOException}
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{CarbonEnv, Row, SQLContext, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.execution.command.{AlterTableModel, AtomicRunnableCommand, CarbonMergerMapping, CompactionModel}
@@ -30,11 +30,11 @@ import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.AlterTableUtil
 
-import org.apache.carbondata.common.exceptions.ConcurrentOperationException
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
@@ -91,16 +91,6 @@ case class CarbonAlterTableCompactionCommand(
   }
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER: LogService =
-      LogServiceFactory.getLogService(this.getClass.getName)
-    val isLoadInProgress = SegmentStatusManager.checkIfAnyLoadInProgressForTable(table)
-    if (isLoadInProgress) {
-      val message = "Cannot run data loading and compaction on same table concurrently. " +
-                    "Please wait for load to finish"
-      LOGGER.error(message)
-      throw new ConcurrentOperationException(message)
-    }
-
     var compactionType: CompactionType = null
     var compactionException = "true"
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
index d2adc57..2092028 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
@@ -26,10 +26,10 @@ import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.carbondata.api.CarbonStore
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.indexstore.PartitionSpec
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.events.{CleanFilesPostEvent, CleanFilesPreEvent, OperationContext, OperationListenerBus}
-import org.apache.carbondata.spark.exception.ConcurrentOperationException
 import org.apache.carbondata.spark.util.CommonUtil
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala
index 0861c63..81427a1 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByIdCommand.scala
@@ -21,9 +21,9 @@ import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.execution.command.{Checker, DataCommand}
 
 import org.apache.carbondata.api.CarbonStore
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.events.{DeleteSegmentByIdPostEvent, DeleteSegmentByIdPreEvent, OperationContext, OperationListenerBus}
-import org.apache.carbondata.spark.exception.ConcurrentOperationException
 
 case class CarbonDeleteLoadByIdCommand(
     loadIds: Seq[String],

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala
index dcbc6ce..1d76bda 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonDeleteLoadByLoadDateCommand.scala
@@ -21,9 +21,9 @@ import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.execution.command.{Checker, DataCommand}
 
 import org.apache.carbondata.api.CarbonStore
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.events.{DeleteSegmentByDatePostEvent, DeleteSegmentByDatePreEvent, OperationContext, OperationListenerBus}
-import org.apache.carbondata.spark.exception.ConcurrentOperationException
 
 case class CarbonDeleteLoadByLoadDateCommand(
     databaseNameOp: Option[String],

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 4806a6f..92e8942 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -44,7 +44,7 @@ import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.types._
 import org.apache.spark.storage.StorageLevel
 import org.apache.spark.unsafe.types.UTF8String
-import org.apache.spark.util.{CarbonReflectionUtils, CausedBy, FileUtils}
+import org.apache.spark.util.{CarbonReflectionUtils, CausedBy, CsvRDDHelper, FileUtils}
 
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
@@ -59,10 +59,9 @@ import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
-import org.apache.carbondata.core.statusmanager.SegmentStatus
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.DataTypeUtil
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.events.exception.PreEventException
@@ -71,18 +70,15 @@ import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.TableProcessingOperations
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
-import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
-import org.apache.carbondata.processing.util.CarbonLoaderUtil
+import org.apache.carbondata.processing.loading.model.{CarbonLoadModelBuilder, LoadOption}
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
 import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
-import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, CarbonDropPartitionRDD}
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, GlobalDictionaryUtil}
 import org.apache.carbondata.spark.load.DataLoadProcessorStepOnSpark
 import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataLoadingUtil, GlobalDictionaryUtil, SparkDataTypeConverterImpl}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil, SparkDataTypeConverterImpl}
 
 case class CarbonLoadDataCommand(
     databaseNameOp: Option[String],
@@ -193,12 +189,18 @@ case class CarbonLoadDataCommand(
       carbonLoadModel.setAggLoadRequest(
         internalOptions.getOrElse(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL, "false").toBoolean)
       carbonLoadModel.setSegmentId(internalOptions.getOrElse("mergedSegmentName", ""))
+
+      val javaPartition = mutable.Map[String, String]()
+      partition.foreach { case (k, v) =>
+        if (v.isEmpty) javaPartition(k) = null else javaPartition(k) = v.get
+      }
+
       new CarbonLoadModelBuilder(table).build(
         options.asJava,
         optionsFinal,
         carbonLoadModel,
         hadoopConf,
-        partition,
+        javaPartition.asJava,
         dataFrame.isDefined)
       // Delete stale segment folders that are not in table status but are physically present in
       // the Fact folder
@@ -231,11 +233,7 @@ case class CarbonLoadDataCommand(
         // First system has to partition the data first and then call the load data
         LOGGER.info(s"Initiating Direct Load for the Table : ($dbName.$tableName)")
         // Clean up the old invalid segment data before creating a new entry for new load.
-        SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, false)
-        DataLoadingUtil.deleteLoadsAndUpdateMetadata(
-          isForceDeletion = false,
-          table,
-          currPartitions)
+        SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, false, currPartitions)
         // add the start entry for the new load in the table status file
         if (updateModel.isEmpty && !table.isHivePartitionTable) {
           CarbonLoaderUtil.readAndUpdateLoadProgressInTableMeta(
@@ -672,7 +670,7 @@ case class CarbonLoadDataCommand(
           }
         }
         val columnCount = carbonLoadModel.getCsvHeaderColumns.length
-        val rdd = DataLoadingUtil.csvFileScanRDD(
+        val rdd = CsvRDDHelper.csvFileScanRDD(
           sparkSession,
           model = carbonLoadModel,
           hadoopConf).map(DataLoadProcessorStepOnSpark.toStringArrayRow(_, columnCount))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
index f074285..230378b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
@@ -22,8 +22,8 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 
-import org.apache.carbondata.common.exceptions.ConcurrentOperationException
 import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
index 5165342..2a92478 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
@@ -25,10 +25,10 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.types.ArrayType
 import org.apache.spark.storage.StorageLevel
 
-import org.apache.carbondata.common.exceptions.ConcurrentOperationException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.Segment
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index 5f8eb12..474f9c6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -23,12 +23,12 @@ import org.apache.spark.sql.execution.command.{AlterTableRenameModel, MetadataCo
 import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionCatalog}
 import org.apache.spark.util.AlterTableUtil
 
-import org.apache.carbondata.common.exceptions.ConcurrentOperationException
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
index 8001a93..0298eea 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
@@ -28,12 +28,12 @@ import org.apache.spark.sql.execution.command.AtomicRunnableCommand
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.cache.dictionary.ManageDictionaryAndBTree
 import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.exception.ConcurrentOperationException
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.events._
-import org.apache.carbondata.spark.exception.{ConcurrentOperationException, ProcessMetaDataException}
 
 case class CarbonDropTableCommand(
     ifExistsSet: Boolean,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
index 61a31a5..2eed988 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
@@ -50,11 +50,8 @@ import org.apache.carbondata.hadoop.api.{CarbonOutputCommitter, CarbonTableOutpu
 import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat.CarbonRecordWriter
 import org.apache.carbondata.hadoop.internal.ObjectArrayWritable
 import org.apache.carbondata.hadoop.util.ObjectSerializationUtil
-import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable
 import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, Util}
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataLoadingUtil, Util}
 
 class CarbonFileFormat
   extends FileFormat

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
index 995f041..d94570a 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
@@ -29,6 +29,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.exception.ProcessMetaDataException
 
 class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
index 1104229..66f8bc5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
@@ -74,7 +74,8 @@ public class DataMapWriterListener {
     }
     List<String> columns = factory.getMeta().getIndexedColumns();
     List<AbstractDataMapWriter> writers = registry.get(columns);
-    AbstractDataMapWriter writer = factory.createWriter(new Segment(segmentId, null), dataWritePath);
+    AbstractDataMapWriter writer = factory.createWriter(
+        new Segment(segmentId, null), dataWritePath);
     if (writers != null) {
       writers.add(writer);
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
index 17e8dbe..29dfa40 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.processing.loading.model;
 
 import java.io.IOException;
 import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -100,6 +102,26 @@ public class CarbonLoadModelBuilder {
       Map<String, String> optionsFinal,
       CarbonLoadModel carbonLoadModel,
       Configuration hadoopConf) throws InvalidLoadOptionException, IOException {
+    build(options, optionsFinal, carbonLoadModel, hadoopConf, new HashMap<String, String>(), false);
+  }
+
+  /**
+   * build CarbonLoadModel for data loading
+   * @param options Load options from user input
+   * @param optionsFinal Load options that populated with default values for optional options
+   * @param carbonLoadModel The output load model
+   * @param hadoopConf hadoopConf is needed to read CSV header if there 'fileheader' is not set in
+   *                   user provided load options
+   * @param partitions partition name map to path
+   * @param isDataFrame true if build for load for dataframe
+   */
+  public void build(
+      Map<String, String> options,
+      Map<String, String> optionsFinal,
+      CarbonLoadModel carbonLoadModel,
+      Configuration hadoopConf,
+      Map<String, String> partitions,
+      boolean isDataFrame) throws InvalidLoadOptionException, IOException {
     carbonLoadModel.setTableName(table.getTableName());
     carbonLoadModel.setDatabaseName(table.getDatabaseName());
     carbonLoadModel.setTablePath(table.getTablePath());
@@ -214,8 +236,18 @@ public class CarbonLoadModelBuilder {
     carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimeter));
     carbonLoadModel.setCsvHeader(fileHeader);
     carbonLoadModel.setColDictFilePath(column_dict);
+
+    List<String> ignoreColumns = new ArrayList<>();
+    if (!isDataFrame) {
+      for (Map.Entry<String, String> partition : partitions.entrySet()) {
+        if (partition.getValue() != null) {
+          ignoreColumns.add(partition.getKey());
+        }
+      }
+    }
+
     carbonLoadModel.setCsvHeaderColumns(
-        LoadOption.getCsvHeaderColumns(carbonLoadModel, hadoopConf));
+        LoadOption.getCsvHeaderColumns(carbonLoadModel, hadoopConf, ignoreColumns));
 
     int validatedMaxColumns = validateMaxColumns(
         carbonLoadModel.getCsvHeaderColumns(),

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
index 5af4859..bac1a94 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.processing.loading.model;
 
 import java.io.IOException;
 import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.carbondata.common.Maps;
@@ -201,6 +203,16 @@ public class LoadOption {
   public static String[] getCsvHeaderColumns(
       CarbonLoadModel carbonLoadModel,
       Configuration hadoopConf) throws IOException {
+    return getCsvHeaderColumns(carbonLoadModel, hadoopConf, new LinkedList<String>());
+  }
+
+  /**
+   * Return CSV header field names, with partition column
+   */
+  public static String[] getCsvHeaderColumns(
+      CarbonLoadModel carbonLoadModel,
+      Configuration hadoopConf,
+      List<String> staticPartitionCols) throws IOException {
     String delimiter;
     if (StringUtils.isEmpty(carbonLoadModel.getCsvDelimiter())) {
       delimiter = CarbonCommonConstants.COMMA;
@@ -231,7 +243,7 @@ public class LoadOption {
     }
 
     if (!CarbonDataProcessorUtil.isHeaderValid(carbonLoadModel.getTableName(), csvColumns,
-        carbonLoadModel.getCarbonDataLoadSchema())) {
+        carbonLoadModel.getCarbonDataLoadSchema(), staticPartitionCols)) {
       if (csvFile == null) {
         LOG.error("CSV header in DDL is not proper."
             + " Column names in schema and CSV header are not the same.");
@@ -249,4 +261,5 @@ public class LoadOption {
     }
     return csvColumns;
   }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index d2faef5..142b2cb 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -21,7 +21,16 @@ import java.io.File;
 import java.io.IOException;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Calendar;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -32,7 +41,6 @@ import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.mutate.DeleteDeltaBlockDetails;
@@ -592,10 +600,6 @@ public final class CarbonDataMergerUtil {
     List<LoadMetadataDetails> segmentsToBeMerged =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
-    CarbonTable carbonTable = carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable();
-    CarbonTableIdentifier tableIdentifier = carbonTable.getCarbonTableIdentifier();
-
-
     // total length
     long totalLength = 0;
 
@@ -1013,7 +1017,8 @@ public final class CarbonDataMergerUtil {
     CarbonFile[] updateDeltaFiles = null;
     Set<String> uniqueBlocks = new HashSet<String>();
 
-    String segmentPath = CarbonTablePath.getSegmentPath(absoluteTableIdentifier.getTablePath(), seg.getSegmentNo());
+    String segmentPath = CarbonTablePath.getSegmentPath(
+        absoluteTableIdentifier.getTablePath(), seg.getSegmentNo());
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
     CarbonFile[] allSegmentFiles = segDir.listFiles();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index ea11e22..ebcf944 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -405,8 +405,8 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
           partitionSpec.getLocation().toString() + CarbonCommonConstants.FILE_SEPARATOR
               + carbonLoadModel.getFactTimeStamp() + ".tmp";
     } else {
-      carbonStoreLocation = CarbonDataProcessorUtil
-          .createCarbonStoreLocation(carbonLoadModel.getDatabaseName(), tableName, carbonLoadModel.getSegmentId());
+      carbonStoreLocation = CarbonDataProcessorUtil.createCarbonStoreLocation(
+          carbonLoadModel.getDatabaseName(), tableName, carbonLoadModel.getSegmentId());
     }
     CarbonFactDataHandlerModel carbonFactDataHandlerModel = CarbonFactDataHandlerModel
         .getCarbonFactDataHandlerModel(carbonLoadModel, carbonTable, segmentProperties, tableName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
index 278d5bb..2616def 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
@@ -76,8 +76,8 @@ public class RowResultMergerProcessor extends AbstractResultProcessor {
           partitionSpec.getLocation().toString() + CarbonCommonConstants.FILE_SEPARATOR + loadModel
               .getFactTimeStamp() + ".tmp";
     } else {
-      carbonStoreLocation = CarbonDataProcessorUtil
-          .createCarbonStoreLocation(loadModel.getDatabaseName(), tableName, loadModel.getSegmentId());
+      carbonStoreLocation = CarbonDataProcessorUtil.createCarbonStoreLocation(
+          loadModel.getDatabaseName(), tableName, loadModel.getSegmentId());
     }
     CarbonFactDataHandlerModel carbonFactDataHandlerModel = CarbonFactDataHandlerModel
         .getCarbonFactDataHandlerModel(loadModel, carbonTable, segProp, tableName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
index df2e2a2..221697f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
@@ -47,8 +47,8 @@ public class RowResultProcessor {
     CarbonDataProcessorUtil.createLocations(tempStoreLocation);
     this.segmentProperties = segProp;
     String tableName = carbonTable.getTableName();
-    String carbonStoreLocation = CarbonDataProcessorUtil
-        .createCarbonStoreLocation(loadModel.getDatabaseName(), tableName, loadModel.getSegmentId());
+    String carbonStoreLocation = CarbonDataProcessorUtil.createCarbonStoreLocation(
+        loadModel.getDatabaseName(), tableName, loadModel.getSegmentId());
     CarbonFactDataHandlerModel carbonFactDataHandlerModel =
         CarbonFactDataHandlerModel.getCarbonFactDataHandlerModel(loadModel, carbonTable,
             segProp, tableName, tempStoreLocation, carbonStoreLocation);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index dc8ffd7..19ad47d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -370,7 +370,8 @@ public final class CarbonDataProcessorUtil {
    *
    * @return data directory path
    */
-  public static String createCarbonStoreLocation(String databaseName, String tableName, String segmentId) {
+  public static String createCarbonStoreLocation(String databaseName, String tableName,
+      String segmentId) {
     CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
     return CarbonTablePath.getSegmentPath(carbonTable.getTablePath(), segmentId);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7466d653/store/sdk/pom.xml
----------------------------------------------------------------------
diff --git a/store/sdk/pom.xml b/store/sdk/pom.xml
index 54fba55..b3dd464 100644
--- a/store/sdk/pom.xml
+++ b/store/sdk/pom.xml
@@ -7,7 +7,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.3.0-SNAPSHOT</version>
+    <version>1.4.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 


[20/49] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
index c8a6b1d..184bf1b 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.OperationContext
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
@@ -74,7 +74,7 @@ class AggregateDataMapCompactor(carbonLoadModel: CarbonLoadModel,
           "true")
         loadCommand.processData(sqlContext.sparkSession)
         val newLoadMetaDataDetails = SegmentStatusManager.readLoadMetadata(
-          carbonTable.getMetaDataFilepath, uuid)
+          carbonTable.getMetadataPath, uuid)
         val updatedLoadMetaDataDetails = newLoadMetaDataDetails collect {
           case load if loadMetaDataDetails.contains(load) =>
             load.setMergedLoadName(mergedLoadName)
@@ -83,11 +83,8 @@ class AggregateDataMapCompactor(carbonLoadModel: CarbonLoadModel,
             load
           case other => other
         }
-        val carbonTablePath = CarbonStorePath
-          .getCarbonTablePath(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-            .getAbsoluteTableIdentifier)
-        SegmentStatusManager
-          .writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePathWithUUID(uuid),
+        SegmentStatusManager.writeLoadDetailsIntoFile(
+          CarbonTablePath.getTableStatusFilePathWithUUID(uuid),
             updatedLoadMetaDataDetails)
         carbonLoadModel.setLoadMetadataDetails(updatedLoadMetaDataDetails.toList.asJava)
       } finally {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 6f08154..8d3110a 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -59,7 +59,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.FailureCauses
@@ -72,7 +72,7 @@ import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonData
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.{DataLoadResultImpl, PartitionFactory, _}
 import org.apache.carbondata.spark.load._
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, DataLoadingUtil, Util}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
 
 /**
  * This is the factory class which can create different RDD depends on user needs.
@@ -127,7 +127,7 @@ object CarbonDataRDDFactory {
       LOGGER.error("Not able to acquire the compaction lock for table " +
           s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
       CarbonCompactionUtil
-          .createCompactionRequiredFile(carbonTable.getMetaDataFilepath, compactionType)
+          .createCompactionRequiredFile(carbonTable.getMetadataPath, compactionType)
       // throw exception only in case of DDL trigger.
       if (compactionModel.isDDLTrigger) {
         CarbonException.analysisException(
@@ -195,7 +195,7 @@ object CarbonDataRDDFactory {
                   s"${ tableForCompaction.getDatabaseName }." +
                   s"${ tableForCompaction.getTableName}")
               val table: CarbonTable = tableForCompaction
-              val metadataPath = table.getMetaDataFilepath
+              val metadataPath = table.getMetadataPath
               val compactionType = CarbonCompactionUtil.determineCompactionType(metadataPath)
 
               val newCarbonLoadModel = prepareCarbonLoadModel(table)
@@ -577,15 +577,13 @@ object CarbonDataRDDFactory {
         (row.get(row.size - 1).toString, Row(row.toSeq.slice(0, row.size - 1): _*)))
 
       val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-        carbonTable.getMetaDataFilepath)
+        carbonTable.getMetadataPath)
         .filter(lmd => lmd.getSegmentStatus.equals(SegmentStatus.LOAD_PARTIAL_SUCCESS) ||
                        lmd.getSegmentStatus.equals(SegmentStatus.SUCCESS))
       val segmentIds = loadMetadataDetails.map(_.getLoadName)
       val segmentIdIndex = segmentIds.zipWithIndex.toMap
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonLoadModel.getTablePath,
-        carbonTable.getCarbonTableIdentifier)
       val segmentId2maxTaskNo = segmentIds.map { segId =>
-        (segId, CarbonUpdateUtil.getLatestTaskIdForSegment(segId, carbonTablePath))
+        (segId, CarbonUpdateUtil.getLatestTaskIdForSegment(segId, carbonLoadModel.getTablePath))
       }.toMap
 
       class SegmentPartitioner(segIdIndex: Map[String, Int], parallelism: Int)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
index 07acaa5..e1bef9c 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
@@ -136,7 +136,7 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
     val validSegments: List[Segment] = CarbonDataMergerUtil.getValidSegments(loadsToMerge)
     val carbonMergerMapping = CarbonMergerMapping(
       tablePath,
-      carbonTable.getMetaDataFilepath,
+      carbonTable.getMetadataPath,
       mergedLoadName,
       databaseName,
       factTableName,
@@ -148,7 +148,7 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
       currentPartitions = partitions)
     carbonLoadModel.setTablePath(carbonMergerMapping.hdfsStoreLocation)
     carbonLoadModel.setLoadMetadataDetails(
-      SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath).toList.asJava)
+      SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath).toList.asJava)
     // trigger event for compaction
     val alterTableCompactionPreEvent: AlterTableCompactionPreEvent =
       AlterTableCompactionPreEvent(sqlContext.sparkSession,
@@ -234,11 +234,11 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
         ((compactionType == CompactionType.IUD_UPDDEL_DELTA) &&
          CarbonDataMergerUtil
            .updateLoadMetadataIUDUpdateDeltaMergeStatus(loadsToMerge,
-             carbonTable.getMetaDataFilepath,
+             carbonTable.getMetadataPath,
              carbonLoadModel)) ||
         CarbonDataMergerUtil.updateLoadMetadataWithMergeStatus(
           loadsToMerge,
-            carbonTable.getMetaDataFilepath,
+            carbonTable.getMetadataPath,
             mergedLoadNumber,
           carbonLoadModel,
           compactionType,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 7e3b699..4645f98 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -34,12 +34,12 @@ import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
-import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events._
 import org.apache.carbondata.processing.loading.events.LoadEvents.LoadMetadataEvent
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
@@ -292,9 +292,10 @@ case class CarbonAlterTableCompactionCommand(
           true)(sparkSession,
           sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog])
         // 5. remove checkpoint
-        val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-        FileFactory.deleteAllFilesOfDir(new File(tablePath.getStreamingCheckpointDir))
-        FileFactory.deleteAllFilesOfDir(new File(tablePath.getStreamingLogDir))
+        FileFactory.deleteAllFilesOfDir(
+          new File(CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath)))
+        FileFactory.deleteAllFilesOfDir(
+          new File(CarbonTablePath.getStreamingLogDir(carbonTable.getTablePath)))
       } else {
         val msg = "Failed to close streaming table, because streaming is locked for table " +
                   carbonTable.getDatabaseName() + "." + carbonTable.getTableName()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 7800d3e..5dbd383 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -60,7 +60,9 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
 import org.apache.carbondata.core.statusmanager.SegmentStatus
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.events.exception.PreEventException
 import org.apache.carbondata.hadoop.util.ObjectSerializationUtil
@@ -258,8 +260,7 @@ case class CarbonLoadDataCommand(
           carbonLoadModel.setUseOnePass(false)
         }
         // Create table and metadata folders if not exist
-        val carbonTablePath = CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
-        val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+        val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
         val fileType = FileFactory.getFileType(metadataDirectoryPath)
         if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
           FileFactory.mkdirs(metadataDirectoryPath, fileType)
@@ -354,9 +355,7 @@ case class CarbonLoadDataCommand(
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
     val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       .getCarbonTableIdentifier
-    val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(carbonLoadModel.getTablePath, carbonTableIdentifier)
-    val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
+    val dictFolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
     val dimensions = carbonTable.getDimensionByTableName(
       carbonTable.getTableName).asScala.toArray
     val colDictFilePath = carbonLoadModel.getColDictFilePath
@@ -1035,4 +1034,5 @@ case class CarbonLoadDataCommand(
     val dataFrameWithTupleId = dataFrame.get.select(fieldWithTupleId: _*)
     (dataFrameWithTupleId)
   }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
index f8f215f..1e5885e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
@@ -45,7 +45,7 @@ case class CarbonShowLoadsCommand(
     val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
     CarbonStore.showSegments(
       limit,
-      carbonTable.getMetaDataFilepath
+      carbonTable.getMetadataPath
     )
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
index e3e4c7a..a8316e9 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala
@@ -36,7 +36,7 @@ import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.{DataMapSchema, TableInfo}
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
 import org.apache.carbondata.hadoop.util.SchemaReader
 
@@ -63,19 +63,18 @@ case class RefreshCarbonTableCommand(
     // 2.2.1 validate that all the aggregate tables are copied at the store location.
     // 2.2.2 Register the aggregate tables
     val tablePath = CarbonEnv.getTablePath(databaseNameOp, tableName)(sparkSession)
-    val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
+    val identifier = AbsoluteTableIdentifier.from(tablePath, databaseName, tableName)
     // 2.1 check if the table already register with hive then ignore and continue with the next
     // schema
     if (!sparkSession.sessionState.catalog.listTables(databaseName)
       .exists(_.table.equalsIgnoreCase(tableName))) {
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
       // check the existence of the schema file to know its a carbon table
-      val schemaFilePath = carbonTablePath.getSchemaFilePath
+      val schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath)
       // if schema file does not exist then the table will either non carbon table or stale
       // carbon table
       if (FileFactory.isFileExist(schemaFilePath, FileFactory.getFileType(schemaFilePath))) {
         // read TableInfo
-        val tableInfo = SchemaReader.getTableInfo(absoluteTableIdentifier)
+        val tableInfo = SchemaReader.getTableInfo(identifier)
         // 2.2 register the table with the hive check if the table being registered has
         // aggregate table then do the below steps
         // 2.2.1 validate that all the aggregate tables are copied at the store location.
@@ -99,7 +98,7 @@ case class RefreshCarbonTableCommand(
         // Register partitions to hive metastore in case of hive partitioning carbon table
         if (tableInfo.getFactTable.getPartitionInfo != null &&
             tableInfo.getFactTable.getPartitionInfo.getPartitionType == PartitionType.NATIVE_HIVE) {
-          registerAllPartitionsToHive(absoluteTableIdentifier, sparkSession)
+          registerAllPartitionsToHive(identifier, sparkSession)
         }
       } else {
         LOGGER.audit(
@@ -178,9 +177,7 @@ case class RefreshCarbonTableCommand(
     dataMapSchemaList.asScala.foreach(dataMap => {
       val tableName = dataMap.getChildSchema.getTableName
       val tablePath = CarbonEnv.getTablePath(Some(dbName), tableName)(sparkSession)
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath,
-        new CarbonTableIdentifier(dbName, tableName, dataMap.getChildSchema.getTableId))
-      val schemaFilePath = carbonTablePath.getSchemaFilePath
+      val schemaFilePath = CarbonTablePath.getSchemaFilePath(tablePath)
       try {
         fileExist = FileFactory.isFileExist(schemaFilePath, FileFactory.getFileType(schemaFilePath))
       } catch {
@@ -191,7 +188,7 @@ case class RefreshCarbonTableCommand(
         return fileExist;
       }
     })
-    return true
+    true
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
index 25d5e91..10d55ef 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
@@ -41,7 +41,7 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, DeleteDeltaBlockDetails, SegmentUpdateDetails, TupleIdEnum}
 import org.apache.carbondata.core.mutate.data.RowCountDetailsVO
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentUpdateStatusManager}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.processing.exception.MultipleMatchingException
@@ -68,12 +68,11 @@ object DeleteExecution {
     val database = CarbonEnv.getDatabaseName(databaseNameOp)(sparkSession)
     val carbonTable = CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
     val isPartitionTable = carbonTable.isHivePartitionTable
     val factPath = if (isPartitionTable) {
-      carbonTablePath.getPath
+      absoluteTableIdentifier.getTablePath
     } else {
-      carbonTablePath.getFactDir
+      CarbonTablePath.getFactDir(absoluteTableIdentifier.getTablePath)
     }
     var segmentsTobeDeleted = Seq.empty[Segment]
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
index 38ac58e..c1f86ef 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
@@ -39,7 +39,6 @@ import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.processing.loading.TableProcessingOperations
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.spark.partition.DropPartitionCallable
@@ -65,8 +64,8 @@ case class CarbonAlterTableDropPartitionCommand(
     if (relation == null || CarbonMetadata.getInstance.getCarbonTable(dbName, tableName) == null) {
       throwMetadataException(dbName, tableName, "table not found")
     }
-    val table = relation.carbonTable
-    val partitionInfo = table.getPartitionInfo(tableName)
+    val carbonTable = relation.carbonTable
+    val partitionInfo = carbonTable.getPartitionInfo(tableName)
     if (partitionInfo == null) {
       throwMetadataException(dbName, tableName, "table is not a partition table")
     }
@@ -92,10 +91,9 @@ case class CarbonAlterTableDropPartitionCommand(
           "Dropping range interval partition is unsupported")
     }
     partitionInfo.dropPartition(partitionIndex)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
-    // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
 
+    // read TableInfo
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)(sparkSession)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl()
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
       dbName, tableName, tablePath)
@@ -108,11 +106,11 @@ case class CarbonAlterTableDropPartitionCommand(
     thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
       .setTime_stamp(System.currentTimeMillis)
     carbonMetaStore.updateTableSchemaForAlter(
-      table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-      table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
       thriftTable,
       null,
-      table.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+      carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     // update the schema modified time
     carbonMetaStore.updateAndTouchSchemasUpdatedTime()
     // sparkSession.catalog.refreshTable(tableName)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
index 7aefbbe..3a5ed2e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
@@ -41,7 +41,6 @@ import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.processing.loading.TableProcessingOperations
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.spark.partition.SplitPartitionCallable
@@ -72,8 +71,8 @@ case class CarbonAlterTableSplitPartitionCommand(
       LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
       throwMetadataException(dbName, tableName, "table not found")
     }
-    val table = relation.carbonTable
-    val partitionInfo = table.getPartitionInfo(tableName)
+    val carbonTable = relation.carbonTable
+    val partitionInfo = carbonTable.getPartitionInfo(tableName)
     val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
     // keep a copy of partitionIdList before update partitionInfo.
     // will be used in partition data scan
@@ -88,9 +87,8 @@ case class CarbonAlterTableSplitPartitionCommand(
 
     updatePartitionInfo(partitionInfo, partitionIds)
 
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier)
     // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTable)(sparkSession)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl()
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
       dbName, tableName, tablePath)
@@ -100,12 +98,12 @@ case class CarbonAlterTableSplitPartitionCommand(
     wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
     val thriftTable =
       schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-    carbonMetaStore
-      .updateTableSchemaForAlter(table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-        table.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-        thriftTable,
-        null,
-        table.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
+    carbonMetaStore.updateTableSchemaForAlter(
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+      thriftTable,
+      null,
+      carbonTable.getAbsoluteTableIdentifier.getTablePath)(sparkSession)
     // update the schema modified time
     carbonMetaStore.updateAndTouchSchemasUpdatedTime()
     Seq.empty

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index 0bee383..9df3241 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -16,11 +16,12 @@
  */
 package org.apache.spark.sql.execution.command.preaaggregate
 
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.collection.JavaConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 
-import org.apache.spark.sql._
-import org.apache.spark.sql.CarbonExpressions.{CarbonSubqueryAlias => SubqueryAlias, MatchCastExpression}
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonEnv, CarbonSession, SparkSession, _}
+import org.apache.spark.sql.CarbonExpressions.{CarbonSubqueryAlias => SubqueryAlias}
+import org.apache.spark.sql.CarbonExpressions.MatchCastExpression
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedFunction, UnresolvedRelation}
 import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, AttributeSeq, Cast, Expression, ExprId, NamedExpression, ScalaUDF}
@@ -35,16 +36,12 @@ import org.apache.spark.sql.types.DataType
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
-import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchema, CarbonTable, DataMapSchema, TableSchema}
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.format.TableInfo
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CommonUtil
 
@@ -425,9 +422,7 @@ object PreAggregateUtil {
       locks = acquireLock(dbName, tableName, locksToBeAcquired, carbonTable)
       // get the latest carbon table and check for column existence
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(
-        carbonTable.getAbsoluteTableIdentifier)
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
         thriftTableInfo,
@@ -516,6 +511,26 @@ object PreAggregateUtil {
     }
   }
 
+  /**
+   * This method reverts the changes to the schema if add column command fails.
+   *
+   * @param dbName
+   * @param tableName
+   * @param numberOfChildSchema
+   * @param sparkSession
+   */
+  def revertMainTableChanges(dbName: String, tableName: String, numberOfChildSchema: Int)
+    (sparkSession: SparkSession): Unit = {
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
+    carbonTable.getTableLastUpdatedTime
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+    if (thriftTable.dataMapSchemas.size > numberOfChildSchema) {
+      metastore.revertTableSchemaForPreAggCreationFailure(
+        carbonTable.getAbsoluteTableIdentifier, thriftTable)(sparkSession)
+    }
+  }
+
   def getChildCarbonTable(databaseName: String, tableName: String)
     (sparkSession: SparkSession): Option[CarbonTable] = {
     val metaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
index 4b43ea7..064ba18 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
@@ -28,7 +28,6 @@ import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.events.{AlterTableAddColumnPostEvent, AlterTableAddColumnPreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.TableInfo
 import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
@@ -64,9 +63,7 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
       OperationListenerBus.getInstance().fireEvent(alterTableAddColumnListener, operationContext)
       // get the latest carbon table and check for column existence
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter
         .fromExternalToWrapperTableInfo(thriftTableInfo,
@@ -76,8 +73,7 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
       newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
         dbName,
         wrapperTableInfo,
-        carbonTablePath,
-        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getAbsoluteTableIdentifier,
         sparkSession.sparkContext).process
       // generate dictionary files for the newly added columns
       new AlterTableAddColumnRDD(sparkSession.sparkContext,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
index 571e23f..f4077e6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
@@ -26,8 +26,9 @@ import org.apache.spark.util.AlterTableUtil
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{AlterTableDataTypeChangePostEvent, AlterTableDataTypeChangePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
@@ -74,9 +75,7 @@ private[sql] case class CarbonAlterTableDataTypeChangeCommand(
         throwMetadataException(dbName, tableName, s"Invalid Column: $columnName")
       }
       // read the latest schema file
-      val carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
       // maintain the added column for schema evolution history
       var addColumnSchema: ColumnSchema = null
       var deletedColumnSchema: ColumnSchema = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
index 780ac8f..7bbefd7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -29,7 +29,7 @@ import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{AlterTableDropColumnPostEvent, AlterTableDropColumnPreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
@@ -99,10 +99,8 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
       OperationListenerBus.getInstance().fireEvent(alterTableDropColumnPreEvent, operationContext)
 
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
       val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+        metastore.getThriftTableInfo(carbonTable)(sparkSession)
       // maintain the deleted columns for schema evolution history
       var deletedColumnSchema = ListBuffer[org.apache.carbondata.format.ColumnSchema]()
       val columnSchemaList = tableInfo.fact_table.table_columns.asScala

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index c8f64e1..a55dbdd 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -32,7 +32,7 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{AlterTableRenamePostEvent, AlterTableRenamePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.exception.{ConcurrentOperationException, MalformedCarbonCommandException}
@@ -95,8 +95,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
       val oldTableIdentifier = carbonTable.getAbsoluteTableIdentifier
       DataMapStoreManager.getInstance().clearDataMaps(oldTableIdentifier)
       // get the latest carbon table and check for column existence
-      val oldTablePath = CarbonStorePath.getCarbonTablePath(oldTableIdentifier)
-      val tableMetadataFile = oldTablePath.getPath
+      val tableMetadataFile = oldTableIdentifier.getTablePath
       val operationContext = new OperationContext
       // TODO: Pass new Table Path in pre-event.
       val alterTableRenamePreEvent: AlterTableRenamePreEvent = AlterTableRenamePreEvent(
@@ -106,7 +105,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
         sparkSession)
       OperationListenerBus.getInstance().fireEvent(alterTableRenamePreEvent, operationContext)
       val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(oldTablePath)(sparkSession)
+        metastore.getThriftTableInfo(carbonTable)(sparkSession)
       val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
       schemaEvolutionEntry.setTableName(newTableName)
       timeStamp = System.currentTimeMillis()
@@ -115,7 +114,8 @@ private[sql] case class CarbonAlterTableRenameCommand(
       val fileType = FileFactory.getFileType(tableMetadataFile)
       val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
         newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
-      var newTablePath = CarbonUtil.getNewTablePath(oldTablePath, newTableIdentifier.getTableName)
+      var newTablePath = CarbonTablePath.getNewTablePath(
+        oldTableIdentifier.getTablePath, newTableIdentifier.getTableName)
       metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
       val hiveClient = sparkSession.sessionState.catalog.asInstanceOf[CarbonSessionCatalog]
         .getClient()
@@ -130,9 +130,9 @@ private[sql] case class CarbonAlterTableRenameCommand(
       // changed the rename order to deal with situation when carbon table and hive table
       // will point to the same tablePath
       if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
-        val rename = FileFactory.getCarbonFile(oldTablePath.getPath, fileType)
-          .renameForce(oldTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
-                       newTableName)
+        val rename = FileFactory.getCarbonFile(oldTableIdentifier.getTablePath, fileType)
+          .renameForce(
+            CarbonTablePath.getNewTablePath(oldTableIdentifier.getTablePath, newTableName))
         if (!rename) {
           renameBadRecords(newTableName, oldTableName, oldDatabaseName)
           sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
@@ -162,13 +162,11 @@ private[sql] case class CarbonAlterTableRenameCommand(
       case e: Exception =>
         LOGGER.error(e, "Rename table failed: " + e.getMessage)
         if (carbonTable != null) {
-          AlterTableUtil
-            .revertRenameTableChanges(oldTableIdentifier,
-              newTableName,
-              carbonTable.getTablePath,
-              carbonTable.getCarbonTableIdentifier.getTableId,
-              timeStamp)(
-              sparkSession)
+          AlterTableUtil.revertRenameTableChanges(
+            newTableName,
+            carbonTable,
+            timeStamp)(
+            sparkSession)
           renameBadRecords(newTableName, oldTableName, oldDatabaseName)
         }
         throwMetadataException(oldDatabaseName, oldTableName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
index b44dc7e..fd09e48 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonFileMetastore.scala
@@ -46,7 +46,8 @@ import org.apache.carbondata.core.metadata.schema
 import org.apache.carbondata.core.metadata.schema.table
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.util.path.CarbonTablePath.getNewTablePath
 import org.apache.carbondata.core.writer.ThriftWriter
 import org.apache.carbondata.events.{LookupRelationPostEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
@@ -209,11 +210,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     val dbName = identifier.getCarbonTableIdentifier.getDatabaseName
     val tableName = identifier.getCarbonTableIdentifier.getTableName
     val tablePath = identifier.getTablePath
-    val carbonTableIdentifier = new CarbonTableIdentifier(dbName.toLowerCase(),
-      tableName.toLowerCase(), UUID.randomUUID().toString)
-    val carbonTablePath =
-      CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier)
-    val tableMetadataFile = carbonTablePath.getSchemaFilePath
+    val tableMetadataFile = CarbonTablePath.getSchemaFilePath(tablePath)
     val fileType = FileFactory.getFileType(tableMetadataFile)
     if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
       val tableUniqueName = CarbonTable.buildUniqueName(dbName, tableName)
@@ -240,13 +237,13 @@ class CarbonFileMetastore extends CarbonMetaStore {
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       schemaEvolutionEntry: SchemaEvolutionEntry,
       tablePath: String) (sparkSession: SparkSession): String = {
-    val absoluteTableIdentifier = AbsoluteTableIdentifier.from(tablePath, oldTableIdentifier)
+    val identifier = AbsoluteTableIdentifier.from(tablePath, oldTableIdentifier)
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     if (schemaEvolutionEntry != null) {
       thriftTableInfo.fact_table.schema_evolution.schema_evolution_history.add(schemaEvolutionEntry)
     }
-    val oldTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
-    val newTablePath = CarbonUtil.getNewTablePath(oldTablePath, newTableIdentifier.getTableName)
+    val newTablePath = CarbonTablePath.getNewTablePath(
+      identifier.getTablePath, newTableIdentifier.getTableName)
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
       thriftTableInfo,
       newTableIdentifier.getDatabaseName,
@@ -341,8 +338,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
   private def createSchemaThriftFile(
       identifier: AbsoluteTableIdentifier,
       thriftTableInfo: TableInfo): String = {
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(identifier)
-    val schemaFilePath = carbonTablePath.getSchemaFilePath
+    val schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath)
     val schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath)
     val fileType = FileFactory.getFileType(schemaMetadataPath)
     if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
@@ -356,7 +352,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     thriftWriter.write(thriftTableInfo)
     thriftWriter.close()
     updateSchemasUpdatedTime(touchSchemaFileSystemTime())
-    carbonTablePath.getPath
+    identifier.getTablePath
   }
 
   protected def addTableCache(
@@ -431,8 +427,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
     (sparkSession: SparkSession) {
     val dbName = absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName
     val tableName = absoluteTableIdentifier.getCarbonTableIdentifier.getTableName
-    val metadataFilePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier)
-      .getMetadataDirectoryPath
+    val metadataFilePath = CarbonTablePath.getMetadataPath(absoluteTableIdentifier.getTablePath)
     val carbonTable = CarbonMetadata.getInstance.getCarbonTable(dbName, tableName)
     if (null != carbonTable) {
       // clear driver B-tree and dictionary cache
@@ -528,9 +523,9 @@ class CarbonFileMetastore extends CarbonMetaStore {
   override def listAllTables(sparkSession: SparkSession): Seq[CarbonTable] =
     metadata.carbonTables
 
-  override def getThriftTableInfo(tablePath: CarbonTablePath)
+  override def getThriftTableInfo(carbonTable: CarbonTable)
     (sparkSession: SparkSession): TableInfo = {
-    val tableMetadataFile = tablePath.getSchemaFilePath
+    val tableMetadataFile = CarbonTablePath.getSchemaFilePath(carbonTable.getTablePath)
     CarbonUtil.readSchemaFile(tableMetadataFile)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
index 16ef38d..5e242b7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonHiveMetaStore.scala
@@ -28,7 +28,7 @@ import org.apache.carbondata.core.metadata.{schema, AbsoluteTableIdentifier, Car
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.format
 import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.util.CarbonSparkUtil
@@ -96,12 +96,8 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     Seq()
   }
 
-  override def getThriftTableInfo(tablePath: CarbonTablePath)
+  override def getThriftTableInfo(carbonTable: CarbonTable)
     (sparkSession: SparkSession): format.TableInfo = {
-    val identifier = tablePath.getCarbonTableIdentifier
-    val relation = lookupRelation(TableIdentifier(identifier.getTableName,
-      Some(identifier.getDatabaseName)))(sparkSession).asInstanceOf[CarbonRelation]
-    val carbonTable = relation.metaData.carbonTable
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
     schemaConverter.fromWrapperToExternalTableInfo(carbonTable.getTableInfo,
       carbonTable.getDatabaseName,
@@ -148,7 +144,8 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       carbonTablePath: String)(sparkSession: SparkSession): String = {
     val schemaConverter = new ThriftWrapperSchemaConverterImpl
-    updateHiveMetaStoreForAlter(newTableIdentifier,
+    updateHiveMetaStoreForAlter(
+      newTableIdentifier,
       oldTableIdentifier,
       thriftTableInfo,
       carbonTablePath,
@@ -163,7 +160,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
       sparkSession: SparkSession,
       schemaConverter: ThriftWrapperSchemaConverterImpl) = {
     val newTablePath =
-      CarbonUtil.getNewTablePath(new Path(oldTablePath), newTableIdentifier.getTableName)
+      CarbonTablePath.getNewTablePath(oldTablePath, newTableIdentifier.getTableName)
     val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
       thriftTableInfo,
       newTableIdentifier.getDatabaseName,

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
index 7bf8536..c9833d0 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
@@ -37,7 +37,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension}
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 
 /**
  * Represents logical plan for one carbon table
@@ -212,10 +212,7 @@ case class CarbonRelation(
         .getValidAndInvalidSegments.getValidSegments.isEmpty) {
         sizeInBytesLocalValue = 0L
       } else {
-        val carbonTablePath = CarbonStorePath.getCarbonTablePath(
-          carbonTable.getTablePath,
-          carbonTable.getCarbonTableIdentifier)
-        val tablePath = carbonTablePath.getPath
+        val tablePath = carbonTable.getTablePath
         val fileType = FileFactory.getFileType(tablePath)
         if (FileFactory.isFileExist(tablePath, fileType)) {
           // get the valid segments

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index 8ebd5a9..bc36e9c 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -36,7 +36,8 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTable
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.util.path.CarbonTablePath.getNewTablePath
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
@@ -176,41 +177,28 @@ object AlterTableUtil {
 
   /**
    * This method reverts the changes to the schema if the rename table command fails.
-   *
-   * @param oldTableIdentifier
-   * @param newTableName
-   * @param timeStamp
-   * @param sparkSession
    */
-  def revertRenameTableChanges(oldTableIdentifier: TableIdentifier,
+  def revertRenameTableChanges(
       newTableName: String,
-      tablePath: String,
-      tableId: String,
+      oldCarbonTable: CarbonTable,
       timeStamp: Long)
     (sparkSession: SparkSession): Unit = {
-    val database = oldTableIdentifier.database.getOrElse(sparkSession.catalog.currentDatabase)
-    val oldCarbonTableIdentifier = new CarbonTableIdentifier(database,
-      oldTableIdentifier.table, tableId)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(tablePath, oldCarbonTableIdentifier)
+    val tablePath = oldCarbonTable.getTablePath
+    val tableId = oldCarbonTable.getCarbonTableIdentifier.getTableId
+    val oldCarbonTableIdentifier = oldCarbonTable.getCarbonTableIdentifier
+    val database = oldCarbonTable.getDatabaseName
     val newCarbonTableIdentifier = new CarbonTableIdentifier(database, newTableName, tableId)
-    val newTablePath = CarbonUtil.getNewTablePath(new Path(tablePath), newTableName)
+    val newTablePath = CarbonTablePath.getNewTablePath(tablePath, newTableName)
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val fileType = FileFactory.getFileType(tablePath)
     if (FileFactory.isFileExist(tablePath, fileType)) {
-      val tableInfo = if (metastore.isReadFromHiveMetaStore) {
-        // In case of hive metastore we first update the carbonschema inside old table only.
-        metastore.getThriftTableInfo(CarbonStorePath.getCarbonTablePath(tablePath,
-          new CarbonTableIdentifier(database, oldTableIdentifier.table, tableId)))(sparkSession)
-      } else {
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      }
+      val tableInfo = metastore.getThriftTableInfo(oldCarbonTable)(sparkSession)
       val evolutionEntryList = tableInfo.fact_table.schema_evolution.schema_evolution_history
       val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
       if (updatedTime == timeStamp) {
-        LOGGER.error(s"Reverting changes for $database.${ oldTableIdentifier.table }")
-        FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
-          .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
-                       oldTableIdentifier.table)
+        LOGGER.error(s"Reverting changes for $database.${oldCarbonTable.getTableName}")
+        FileFactory.getCarbonFile(tablePath, fileType)
+          .renameForce(CarbonTablePath.getNewTablePath(tablePath, oldCarbonTable.getTableName))
         val absoluteTableIdentifier = AbsoluteTableIdentifier.from(
           newTablePath,
           newCarbonTableIdentifier)
@@ -233,9 +221,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
-      carbonTable.getCarbonTableIdentifier)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -260,9 +246,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
-      carbonTable.getCarbonTableIdentifier)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -293,9 +277,7 @@ object AlterTableUtil {
     (sparkSession: SparkSession): Unit = {
     val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
     val carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
-      carbonTable.getCarbonTableIdentifier)
-    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val thriftTable: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
     val evolutionEntryList = thriftTable.fact_table.schema_evolution.schema_evolution_history
     val updatedTime = evolutionEntryList.get(evolutionEntryList.size() - 1).time_stamp
     if (updatedTime == timeStamp) {
@@ -344,9 +326,7 @@ object AlterTableUtil {
       carbonTable = CarbonEnv.getCarbonTable(Some(dbName), tableName)(sparkSession)
       // get the latest carbon table
       // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath,
-        carbonTable.getCarbonTableIdentifier)
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
       val schemaConverter = new ThriftWrapperSchemaConverterImpl()
       val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
         thriftTableInfo,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
index aadee81..0bdef8a 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
@@ -856,9 +856,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
   }
 
   def getDataFiles(carbonTable: CarbonTable, segmentId: String): Array[CarbonFile] = {
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
index e3678cd..1d41ddc 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
@@ -24,7 +24,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.util.TableOptionConstant
 
@@ -65,9 +65,7 @@ class AllDictionaryTestCase extends Spark2QueryTest with BeforeAndAfterAll {
     carbonLoadModel.setCsvHeaderColumns(
       CommonUtil.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
     // Create table and metadata folders if not exist
-    val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(table.getTablePath, table.getCarbonTableIdentifier)
-    val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+    val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
     val fileType = FileFactory.getFileType(metadataDirectoryPath)
     if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
       FileFactory.mkdirs(metadataDirectoryPath, fileType)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
index f9519f8..a465251 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql.test.TestQueryExecutor
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index e543893..7ca0b56 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -26,7 +26,7 @@ import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.util.TableOptionConstant
@@ -179,9 +179,7 @@ class ExternalColumnDictionaryTestCase extends Spark2QueryTest with BeforeAndAft
       CommonUtil.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
     carbonLoadModel.setMaxColumns("100")
     // Create table and metadata folders if not exist
-    val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(table.getTablePath, table.getCarbonTableIdentifier)
-    val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+    val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
     val fileType = FileFactory.getFileType(metadataDirectoryPath)
     if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
       FileFactory.mkdirs(metadataDirectoryPath, fileType)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index 5644302..017bca8 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -33,10 +33,14 @@ import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{FileFormat, SegmentStatus}
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.spark.exception.{MalformedCarbonCommandException, ProcessMetaDataException}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.exception.ProcessMetaDataException
 
 class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
 
@@ -197,7 +201,6 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val identifier = new TableIdentifier("batch_table", Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
       .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     var server: ServerSocket = null
     try {
       server = getServerSocket
@@ -205,7 +208,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
       thread1.start()
       // use thread pool to catch the exception of sink thread
       val pool = Executors.newSingleThreadExecutor()
-      val thread2 = createSocketStreamingThread(spark, server.getLocalPort, tablePath, identifier)
+      val thread2 = createSocketStreamingThread(spark, server.getLocalPort, carbonTable, identifier)
       val future = pool.submit(thread2)
       Thread.sleep(1000)
       thread1.interrupt()
@@ -225,11 +228,10 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val identifier = new TableIdentifier("stream_table_file", Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
       .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     val csvDataDir = new File("target/csvdata").getCanonicalPath
     // streaming ingest 10 rows
     generateCSVDataFile(spark, idStart = 10, rowNums = 10, csvDataDir)
-    val thread = createFileStreamingThread(spark, tablePath, csvDataDir, intervalSecond = 1,
+    val thread = createFileStreamingThread(spark, carbonTable, csvDataDir, intervalSecond = 1,
       identifier)
     thread.start()
     Thread.sleep(2000)
@@ -1086,12 +1088,11 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val identifier = new TableIdentifier("stream_table_drop", Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
       .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     var server: ServerSocket = null
     try {
       server = getServerSocket
       val thread1 = createWriteSocketThread(server, 2, 10, 3)
-      val thread2 = createSocketStreamingThread(spark, server.getLocalPort, tablePath, identifier, "force", 5, 1024L * 200, false)
+      val thread2 = createSocketStreamingThread(spark, server.getLocalPort, carbonTable, identifier, "force", 5, 1024L * 200, false)
       thread1.start()
       thread2.start()
       Thread.sleep(1000)
@@ -1200,7 +1201,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
   def createSocketStreamingThread(
       spark: SparkSession,
       port: Int,
-      tablePath: CarbonTablePath,
+      carbonTable: CarbonTable,
       tableIdentifier: TableIdentifier,
       badRecordAction: String = "force",
       intervalSecond: Int = 2,
@@ -1221,7 +1222,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
           qry = readSocketDF.writeStream
             .format("carbondata")
             .trigger(ProcessingTime(s"$intervalSecond seconds"))
-            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("checkpointLocation", CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath))
             .option("bad_records_action", badRecordAction)
             .option("dbName", tableIdentifier.database.get)
             .option("tableName", tableIdentifier.table)
@@ -1260,7 +1261,6 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
     val identifier = new TableIdentifier(tableName, Option("streaming"))
     val carbonTable = CarbonEnv.getInstance(spark).carbonMetastore.lookupRelation(identifier)(spark)
       .asInstanceOf[CarbonRelation].metaData.carbonTable
-    val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
     var server: ServerSocket = null
     try {
       server = getServerSocket()
@@ -1273,7 +1273,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
       val thread2 = createSocketStreamingThread(
         spark = spark,
         port = server.getLocalPort,
-        tablePath = tablePath,
+        carbonTable = carbonTable,
         tableIdentifier = identifier,
         badRecordAction = badRecordAction,
         intervalSecond = intervalOfIngest,
@@ -1321,7 +1321,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
 
   def createFileStreamingThread(
       spark: SparkSession,
-      tablePath: CarbonTablePath,
+      carbonTable: CarbonTable,
       csvDataDir: String,
       intervalSecond: Int,
       tableIdentifier: TableIdentifier): Thread = {
@@ -1335,7 +1335,7 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
           qry = readSocketDF.writeStream
             .format("carbondata")
             .trigger(ProcessingTime(s"${ intervalSecond } seconds"))
-            .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+            .option("checkpointLocation", CarbonTablePath.getStreamingCheckpointDir(carbonTable.getTablePath))
             .option("dbName", tableIdentifier.database.get)
             .option("tableName", tableIdentifier.table)
             .option("timestampformat", CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
index 4d5f88c..27ed1bd 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableRevertTestCase.scala
@@ -94,7 +94,7 @@ class AlterTableRevertTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       }
       val carbonTable = CarbonMetadata.getInstance.getCarbonTable("default", "reverttest")
 
-      assert(new File(carbonTable.getMetaDataFilepath).listFiles().length < 6)
+      assert(new File(carbonTable.getMetadataPath).listFiles().length < 6)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java b/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
index a8db6c9..bbc3697 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
@@ -34,7 +34,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
@@ -56,43 +55,39 @@ public class TableProcessingOperations {
    */
   public static void deletePartialLoadDataIfExist(CarbonTable carbonTable,
       final boolean isCompactionFlow) throws IOException {
-    String metaDataLocation = carbonTable.getMetaDataFilepath();
+    String metaDataLocation = carbonTable.getMetadataPath();
     final LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metaDataLocation);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(carbonTable.getTablePath(), carbonTable.getCarbonTableIdentifier());
 
     //delete folder which metadata no exist in tablestatus
-    for (int i = 0; i < carbonTable.getPartitionCount(); i++) {
-      String partitionPath = carbonTablePath.getPartitionDir();
-      FileFactory.FileType fileType = FileFactory.getFileType(partitionPath);
-      if (FileFactory.isFileExist(partitionPath, fileType)) {
-        CarbonFile carbonFile = FileFactory.getCarbonFile(partitionPath, fileType);
-        CarbonFile[] listFiles = carbonFile.listFiles(new CarbonFileFilter() {
-          @Override public boolean accept(CarbonFile path) {
-            String segmentId =
-                CarbonTablePath.DataPathUtil.getSegmentId(path.getAbsolutePath() + "/dummy");
-            boolean found = false;
-            for (int j = 0; j < details.length; j++) {
-              if (details[j].getLoadName().equals(segmentId)) {
-                found = true;
-                break;
-              }
-            }
-            return !found;
-          }
-        });
-        for (int k = 0; k < listFiles.length; k++) {
+    String partitionPath = CarbonTablePath.getPartitionDir(carbonTable.getTablePath());
+    FileFactory.FileType fileType = FileFactory.getFileType(partitionPath);
+    if (FileFactory.isFileExist(partitionPath, fileType)) {
+      CarbonFile carbonFile = FileFactory.getCarbonFile(partitionPath, fileType);
+      CarbonFile[] listFiles = carbonFile.listFiles(new CarbonFileFilter() {
+        @Override public boolean accept(CarbonFile path) {
           String segmentId =
-              CarbonTablePath.DataPathUtil.getSegmentId(listFiles[k].getAbsolutePath() + "/dummy");
-          if (isCompactionFlow) {
-            if (segmentId.contains(".")) {
-              CarbonLoaderUtil.deleteStorePath(listFiles[k].getAbsolutePath());
-            }
-          } else {
-            if (!segmentId.contains(".")) {
-              CarbonLoaderUtil.deleteStorePath(listFiles[k].getAbsolutePath());
+              CarbonTablePath.DataFileUtil.getSegmentId(path.getAbsolutePath() + "/dummy");
+          boolean found = false;
+          for (int j = 0; j < details.length; j++) {
+            if (details[j].getLoadName().equals(segmentId)) {
+              found = true;
+              break;
             }
           }
+          return !found;
+        }
+      });
+      for (int k = 0; k < listFiles.length; k++) {
+        String segmentId =
+            CarbonTablePath.DataFileUtil.getSegmentId(listFiles[k].getAbsolutePath() + "/dummy");
+        if (isCompactionFlow) {
+          if (segmentId.contains(".")) {
+            CarbonLoaderUtil.deleteStorePath(listFiles[k].getAbsolutePath());
+          }
+        } else {
+          if (!segmentId.contains(".")) {
+            CarbonLoaderUtil.deleteStorePath(listFiles[k].getAbsolutePath());
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
index 4cd5014..193d192 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
@@ -34,8 +34,6 @@ import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.datatypes.ArrayDataType;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
@@ -105,12 +103,11 @@ public class FieldEncoderFactory {
           ColumnIdentifier parentColumnIdentifier =
               new ColumnIdentifier(parentColumnTableRelation.getColumnId(), null,
                   dataField.getColumn().getDataType());
-          CarbonTablePath carbonTablePath =
-              CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
           AbsoluteTableIdentifier parentAbsoluteTableIdentifier =
               AbsoluteTableIdentifier.from(
-              CarbonUtil.getNewTablePath(carbonTablePath, parentTableIdentifier.getTableName()),
-              parentTableIdentifier);
+                  CarbonTablePath.getNewTablePath(
+                      absoluteTableIdentifier.getTablePath(), parentTableIdentifier.getTableName()),
+                  parentTableIdentifier);
           identifier = new DictionaryColumnUniqueIdentifier(parentAbsoluteTableIdentifier,
               parentColumnIdentifier, dataField.getColumn().getDataType());
           return new DictionaryFieldConverterImpl(dataField, cache, parentAbsoluteTableIdentifier,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java
index d3caa99..a08177a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/AbstractResultProcessor.java
@@ -19,10 +19,8 @@ package org.apache.carbondata.processing.merger;
 
 import java.util.List;
 
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
 import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
@@ -42,13 +40,11 @@ public abstract class AbstractResultProcessor {
   public abstract boolean execute(List<RawResultIterator> resultIteratorList);
 
   protected void setDataFileAttributesInModel(CarbonLoadModel loadModel,
-      CompactionType compactionType, CarbonTable carbonTable,
-      CarbonFactDataHandlerModel carbonFactDataHandlerModel) {
+      CompactionType compactionType, CarbonFactDataHandlerModel carbonFactDataHandlerModel) {
     CarbonDataFileAttributes carbonDataFileAttributes;
     if (compactionType == CompactionType.IUD_UPDDEL_DELTA) {
       long taskNo = CarbonUpdateUtil.getLatestTaskIdForSegment(loadModel.getSegmentId(),
-          CarbonStorePath.getCarbonTablePath(loadModel.getTablePath(),
-              carbonTable.getCarbonTableIdentifier()));
+          loadModel.getTablePath());
       // Increase the Task Index as in IUD_UPDDEL_DELTA_COMPACTION the new file will
       // be written in same segment. So the TaskNo should be incremented by 1 from max val.
       long index = taskNo + 1;


[22/49] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Posted by ja...@apache.org.
[CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Refactory CarbonTablePath:

1.Remove CarbonStorePath and use CarbonTablePath only.
2.Make CarbonTablePath an utility without object creation, it can avoid creating object before using it, thus code is cleaner and GC is less.

This closes #1768


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

Branch: refs/heads/carbonstore-rebase4
Commit: 4d453d4b07389b515abff034d3b7ff06ec64abdb
Parents: 975725a
Author: Jacky Li <ja...@qq.com>
Authored: Wed Jan 31 16:14:27 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 00:20:44 2018 +0800

----------------------------------------------------------------------
 .../DictionaryColumnUniqueIdentifier.java       |  29 +-
 .../dictionary/ManageDictionaryAndBTree.java    |  13 +-
 .../core/metadata/AbsoluteTableIdentifier.java  |   4 +-
 .../core/metadata/schema/table/CarbonTable.java |  11 +-
 .../core/mutate/CarbonUpdateUtil.java           |  58 ++--
 .../core/scan/executor/util/QueryUtil.java      |   7 +-
 .../core/service/CarbonCommonFactory.java       |  16 -
 .../carbondata/core/service/PathService.java    |  35 ---
 .../core/service/impl/PathFactory.java          |  50 ----
 .../statusmanager/SegmentStatusManager.java     |  30 +-
 .../SegmentUpdateStatusManager.java             |  59 ++--
 .../apache/carbondata/core/util/CarbonUtil.java |  60 ++--
 .../util/path/CarbonSharedDictionaryPath.java   |  71 -----
 .../core/util/path/CarbonStorePath.java         |  71 -----
 .../core/util/path/CarbonTablePath.java         | 291 ++++++-------------
 .../dictionary/AbstractDictionaryCacheTest.java |  11 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |   6 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |   6 +-
 .../reader/CarbonDictionaryReaderImplTest.java  |   8 -
 .../CarbonFormatDirectoryStructureTest.java     |  18 +-
 .../path/CarbonFormatSharedDictionaryTest.java  |  44 ---
 .../writer/CarbonDictionaryWriterImplTest.java  |  19 +-
 .../CarbonBatchSparkStreamingExample.scala      |   9 +-
 .../CarbonStructuredStreamingExample.scala      |  11 +-
 .../hadoop/api/CarbonTableInputFormat.java      |   8 +-
 .../streaming/CarbonStreamRecordWriter.java     |   6 +-
 .../carbondata/hadoop/util/SchemaReader.java    |  18 +-
 .../hadoop/test/util/StoreCreator.java          |  16 +-
 .../presto/CarbondataRecordSetProvider.java     |   7 +-
 .../presto/impl/CarbonTableCacheModel.java      |  13 +-
 .../presto/impl/CarbonTableReader.java          |  49 ++--
 .../presto/util/CarbonDataStoreCreator.scala    |  10 +-
 .../sdv/generated/MergeIndexTestCase.scala      |   8 +-
 .../dataload/TestLoadDataGeneral.scala          |  11 +-
 .../InsertIntoCarbonTableTestCase.scala         |   8 +-
 .../createTable/TestCreateTableAsSelect.scala   |   2 +-
 .../datacompaction/DataCompactionLockTest.scala |   6 +-
 .../MajorCompactionIgnoreInMinorTest.scala      |  12 +-
 .../dataload/TestBatchSortDataLoad.scala        |   5 +-
 .../dataload/TestDataLoadWithFileName.scala     |   5 +-
 .../dataload/TestGlobalSortDataLoad.scala       |   8 +-
 .../dataretention/DataRetentionTestCase.scala   |   6 +-
 .../TestDataLoadingForPartitionTable.scala      |   4 +-
 .../StandardPartitionTableLoadingTestCase.scala |   6 +-
 .../org/apache/carbondata/api/CarbonStore.scala |   4 +-
 .../carbondata/spark/CarbonSparkFactory.scala   |   2 +-
 .../spark/DictionaryDetailHelper.scala          |   9 +-
 .../spark/rdd/AlterTableAddColumnRDD.scala      |  17 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |   6 -
 .../carbondata/spark/util/CommonUtil.scala      |  30 +-
 .../carbondata/spark/util/DataLoadingUtil.scala |   8 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  16 +-
 .../command/carbonTableSchemaCommon.scala       |   9 +-
 .../org/apache/spark/util/PartitionUtils.scala  |   6 +-
 .../spark/rdd/AggregateDataMapCompactor.scala   |  11 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  14 +-
 .../spark/rdd/CarbonTableCompactor.scala        |   8 +-
 .../CarbonAlterTableCompactionCommand.scala     |  13 +-
 .../management/CarbonLoadDataCommand.scala      |  12 +-
 .../management/CarbonShowLoadsCommand.scala     |   2 +-
 .../management/RefreshCarbonTableCommand.scala  |  17 +-
 .../command/mutation/DeleteExecution.scala      |   7 +-
 .../CarbonAlterTableDropPartitionCommand.scala  |  16 +-
 .../CarbonAlterTableSplitPartitionCommand.scala |  20 +-
 .../preaaggregate/PreAggregateUtil.scala        |  35 ++-
 .../CarbonAlterTableAddColumnCommand.scala      |   8 +-
 .../CarbonAlterTableDataTypeChangeCommand.scala |   9 +-
 .../CarbonAlterTableDropColumnCommand.scala     |   6 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |  28 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |  27 +-
 .../spark/sql/hive/CarbonHiveMetaStore.scala    |  13 +-
 .../apache/spark/sql/hive/CarbonMetaStore.scala |   2 +-
 .../apache/spark/sql/hive/CarbonRelation.scala  |   7 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |  54 ++--
 .../partition/TestAlterPartitionTable.scala     |   4 +-
 .../spark/util/AllDictionaryTestCase.scala      |   6 +-
 .../spark/util/DictionaryTestCaseUtil.scala     |   2 +-
 .../util/ExternalColumnDictionaryTestCase.scala |   6 +-
 .../TestStreamingTableOperation.scala           |  26 +-
 .../restructure/AlterTableRevertTestCase.scala  |   2 +-
 .../loading/TableProcessingOperations.java      |  59 ++--
 .../converter/impl/FieldEncoderFactory.java     |   9 +-
 .../merger/AbstractResultProcessor.java         |   8 +-
 .../processing/merger/CarbonCompactionUtil.java |   2 +-
 .../processing/merger/CarbonDataMergerUtil.java |  76 ++---
 .../store/CarbonFactDataHandlerModel.java       |   6 +-
 .../util/CarbonDataProcessorUtil.java           |  16 +-
 .../processing/util/CarbonLoaderUtil.java       |  93 +++---
 .../processing/util/DeleteLoadFolders.java      |   8 +-
 .../carbon/datastore/BlockIndexStoreTest.java   | 158 +---------
 .../carbondata/processing/StoreCreator.java     |  34 +--
 .../streaming/segment/StreamSegment.java        |  24 +-
 .../carbondata/streaming/StreamHandoffRDD.scala |  22 +-
 .../streaming/StreamSinkFactory.scala           |  14 +-
 .../streaming/CarbonAppendableStreamSink.scala  |  18 +-
 95 files changed, 655 insertions(+), 1499 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
index f5652ac..a7b47c9 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
@@ -22,7 +22,6 @@ import java.io.Serializable;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
@@ -42,8 +41,6 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    */
   private ColumnIdentifier columnIdentifier;
 
-  private transient CarbonTablePath carbonTablePath;
-
   private DataType dataType;
 
   private String dictionaryLocation;
@@ -68,8 +65,7 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
     this.columnIdentifier = columnIdentifier;
     this.dataType = columnIdentifier.getDataType();
     this.dictionaryLocation =
-        CarbonStorePath.getCarbonTablePath(dictionarySourceAbsoluteTableIdentifier)
-            .getMetadataDirectoryPath();
+        CarbonTablePath.getMetadataPath(dictionarySourceAbsoluteTableIdentifier.getTablePath());
   }
 
   /**
@@ -100,13 +96,6 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
     return dataType;
   }
 
-  public CarbonTablePath getCarbonTablePath() {
-    if (null == carbonTablePath) {
-      carbonTablePath = CarbonStorePath.getCarbonTablePath(dictionarySourceAbsoluteTableIdentifier);
-    }
-    return carbonTablePath;
-  }
-
   /**
    * @return columnIdentifier
    */
@@ -118,24 +107,24 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    * @return dictionary file path
    */
   public String getDictionaryFilePath() {
-    return getCarbonTablePath()
-        .getDictionaryFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+    return CarbonTablePath.getExternalDictionaryFilePath(
+        dictionaryLocation, columnIdentifier.getColumnId());
   }
 
   /**
    * @return dictionary metadata file path
    */
   public String getDictionaryMetaFilePath() {
-    return getCarbonTablePath()
-        .getDictionaryMetaFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+    return CarbonTablePath.getExternalDictionaryMetaFilePath(
+        dictionaryLocation, columnIdentifier.getColumnId());
   }
 
   /**
    * @return sort index file path
    */
   public String getSortIndexFilePath() {
-    return getCarbonTablePath()
-        .getSortIndexFilePath(dictionaryLocation, columnIdentifier.getColumnId());
+    return CarbonTablePath.getExternalSortIndexFilePath(
+        dictionaryLocation, columnIdentifier.getColumnId());
   }
 
   /**
@@ -143,8 +132,8 @@ public class DictionaryColumnUniqueIdentifier implements Serializable {
    * @return sort index file path with given offset
    */
   public String getSortIndexFilePath(long offset) {
-    return getCarbonTablePath()
-        .getSortIndexFilePath(dictionaryLocation, columnIdentifier.getColumnId(), offset);
+    return CarbonTablePath.getExternalSortIndexFilePath(
+        dictionaryLocation, columnIdentifier.getColumnId(), offset);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
index 4f8f724..a7d6027 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
@@ -36,7 +36,6 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 /**
@@ -56,13 +55,11 @@ public class ManageDictionaryAndBTree {
    * clear the dictionary cache
    *
    * @param columnSchema
-   * @param carbonTableIdentifier
+   * @param identifier
    */
   public static void deleteDictionaryFileAndCache(final ColumnSchema columnSchema,
-      AbsoluteTableIdentifier carbonTableIdentifier) {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTableIdentifier);
-    String metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath();
+      AbsoluteTableIdentifier identifier) {
+    String metadataDirectoryPath = CarbonTablePath.getMetadataPath(identifier.getTablePath());
     CarbonFile metadataDir = FileFactory
         .getCarbonFile(metadataDirectoryPath, FileFactory.getFileType(metadataDirectoryPath));
     if (metadataDir.exists()) {
@@ -90,7 +87,7 @@ public class ManageDictionaryAndBTree {
       }
     }
     // remove dictionary cache
-    removeDictionaryColumnFromCache(carbonTableIdentifier, columnSchema.getColumnUniqueId());
+    removeDictionaryColumnFromCache(identifier, columnSchema.getColumnUniqueId());
   }
 
   /**
@@ -101,7 +98,7 @@ public class ManageDictionaryAndBTree {
   public static void clearBTreeAndDictionaryLRUCache(CarbonTable carbonTable) {
     // clear Btree cache from LRU cache
     LoadMetadataDetails[] loadMetadataDetails =
-        SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath());
+        SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
     String[] segments = new String[loadMetadataDetails.length];
     int i = 0;
     for (LoadMetadataDetails loadMetadataDetail : loadMetadataDetails) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
index d3250aa..3ea1f60 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/AbsoluteTableIdentifier.java
@@ -36,7 +36,6 @@ public class AbsoluteTableIdentifier implements Serializable {
    */
   private String tablePath;
 
-  private boolean isLocalPath;
 
   /**
    * carbon table identifier which will have table name and table database
@@ -47,7 +46,6 @@ public class AbsoluteTableIdentifier implements Serializable {
   private AbsoluteTableIdentifier(String tablePath, CarbonTableIdentifier carbonTableIdentifier) {
     //TODO this should be moved to common place where path handling will be handled
     this.tablePath = FileFactory.getUpdatedFilePath(tablePath);
-    isLocalPath = tablePath.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX);
     this.carbonTableIdentifier = carbonTableIdentifier;
   }
 
@@ -79,7 +77,7 @@ public class AbsoluteTableIdentifier implements Serializable {
   }
 
   public String appendWithLocalPrefix(String path) {
-    if (isLocalPath) {
+    if (tablePath.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX)) {
       return CarbonCommonConstants.LOCAL_FILE_PREFIX + path;
     } else {
       return path;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index d17d865..6ed1197 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -424,13 +424,20 @@ public class CarbonTable implements Serializable {
   }
 
   /**
-   * @return the metaDataFilepath
+   * Return the metadata path of the table
    */
-  public String getMetaDataFilepath() {
+  public String getMetadataPath() {
     return CarbonTablePath.getMetadataPath(getTablePath());
   }
 
   /**
+   * Return the segment path of the specified segmentId
+   */
+  public String getSemgentPath(String segmentId) {
+    return CarbonTablePath.getSegmentPath(getTablePath(), segmentId);
+  }
+
+  /**
    * @return storepath
    */
   public String getTablePath() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
index 7289c98..4ff19cb 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
@@ -43,10 +43,8 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
-
 /**
  * This class contains all update utility methods
  */
@@ -202,18 +200,10 @@ public class CarbonUpdateUtil {
       List<Segment> segmentsToBeDeleted, List<Segment> segmentFilesTobeUpdated) {
 
     boolean status = false;
-
-    String metaDataFilepath = table.getMetaDataFilepath();
-
-    AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-                    absoluteTableIdentifier.getCarbonTableIdentifier());
-
-    String tableStatusPath = carbonTablePath.getTableStatusFilePath();
-
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    String metaDataFilepath = table.getMetadataPath();
+    AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     boolean lockStatus = false;
@@ -225,7 +215,7 @@ public class CarbonUpdateUtil {
                         + " for table status updation");
 
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-                segmentStatusManager.readLoadMetadata(metaDataFilepath);
+                SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
         for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
 
@@ -312,12 +302,8 @@ public class CarbonUpdateUtil {
    */
   public static void cleanStaleDeltaFiles(CarbonTable table, final String timeStamp) {
 
-    AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-                    absoluteTableIdentifier.getCarbonTableIdentifier());
-    String partitionDir = carbonTablePath.getPartitionDir();
+    AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
+    String partitionDir = CarbonTablePath.getPartitionDir(identifier.getTablePath());
     CarbonFile file =
             FileFactory.getCarbonFile(partitionDir, FileFactory.getFileType(partitionDir));
     if (!file.exists()) {
@@ -387,8 +373,18 @@ public class CarbonUpdateUtil {
         .substring(0, completeBlockName.lastIndexOf(CarbonCommonConstants.HYPHEN));
   }
 
-  public static long getLatestTaskIdForSegment(String segmentId, CarbonTablePath tablePath) {
-    String segmentDirPath = tablePath.getCarbonDataDirectoryPath(segmentId);
+  /**
+   * returns segment id from segment name
+   *
+   * @param segmentName
+   * @return
+   */
+  public static String getSegmentId(String segmentName) {
+    return segmentName.split(CarbonCommonConstants.UNDERSCORE)[1];
+  }
+
+  public static long getLatestTaskIdForSegment(String segmentId, String tablePath) {
+    String segmentDirPath = CarbonTablePath.getSegmentPath(tablePath, segmentId);
 
     // scan all the carbondata files and get the latest task ID.
     CarbonFile segment =
@@ -427,11 +423,8 @@ public class CarbonUpdateUtil {
 
     SegmentStatusManager ssm = new SegmentStatusManager(table.getAbsoluteTableIdentifier());
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(table.getAbsoluteTableIdentifier().getTablePath(),
-                    table.getAbsoluteTableIdentifier().getCarbonTableIdentifier());
-
-    LoadMetadataDetails[] details = ssm.readLoadMetadata(table.getMetaDataFilepath());
+    LoadMetadataDetails[] details =
+        SegmentStatusManager.readLoadMetadata(table.getMetadataPath());
 
     String validUpdateStatusFile = "";
 
@@ -453,7 +446,8 @@ public class CarbonUpdateUtil {
               || segment.getSegmentStatus() == SegmentStatus.LOAD_PARTIAL_SUCCESS) {
 
         // take the list of files from this segment.
-        String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segment.getLoadName());
+        String segmentPath = CarbonTablePath.getSegmentPath(
+            table.getAbsoluteTableIdentifier().getTablePath(), segment.getLoadName());
         CarbonFile segDir =
                 FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
         CarbonFile[] allSegmentFiles = segDir.listFiles();
@@ -560,8 +554,10 @@ public class CarbonUpdateUtil {
       final String updateStatusTimestamp = validUpdateStatusFile
               .substring(validUpdateStatusFile.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1);
 
-      CarbonFile metaFolder = FileFactory.getCarbonFile(carbonTablePath.getMetadataDirectoryPath(),
-              FileFactory.getFileType(carbonTablePath.getMetadataDirectoryPath()));
+      String tablePath = table.getAbsoluteTableIdentifier().getTablePath();
+      CarbonFile metaFolder = FileFactory.getCarbonFile(
+          CarbonTablePath.getMetadataPath(tablePath),
+          FileFactory.getFileType(CarbonTablePath.getMetadataPath(tablePath)));
 
       CarbonFile[] invalidUpdateStatusFiles = metaFolder.listFiles(new CarbonFileFilter() {
         @Override public boolean accept(CarbonFile file) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index 00cdfe7..1765efa 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -63,7 +63,6 @@ import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnRes
 import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import org.apache.commons.lang3.ArrayUtils;
@@ -409,7 +408,7 @@ public class QueryUtil {
   }
 
   public static AbsoluteTableIdentifier getTableIdentifierForColumn(CarbonDimension carbonDimension,
-      AbsoluteTableIdentifier absoluteTableIdentifier) {
+      AbsoluteTableIdentifier identifier) {
     RelationIdentifier relation = carbonDimension.getColumnSchema()
         .getParentColumnTableRelations()
         .get(0)
@@ -417,8 +416,8 @@ public class QueryUtil {
     String parentTableName = relation.getTableName();
     String parentDatabaseName = relation.getDatabaseName();
     String parentTableId = relation.getTableId();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String newTablePath = CarbonUtil.getNewTablePath(carbonTablePath, parentTableName);
+    String newTablePath =
+        CarbonTablePath.getNewTablePath(identifier.getTablePath(), parentTableName);
     return AbsoluteTableIdentifier.from(newTablePath, parentDatabaseName, parentTableName,
         parentTableId);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java b/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
index 6fd9735..3ddf00b 100644
--- a/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/service/CarbonCommonFactory.java
@@ -16,9 +16,7 @@
  */
 package org.apache.carbondata.core.service;
 
-import org.apache.carbondata.core.service.impl.ColumnUniqueIdGenerator;
 import org.apache.carbondata.core.service.impl.DictionaryFactory;
-import org.apache.carbondata.core.service.impl.PathFactory;
 
 /**
  * Interface to get services
@@ -32,18 +30,4 @@ public class CarbonCommonFactory {
     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/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/service/PathService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/service/PathService.java b/core/src/main/java/org/apache/carbondata/core/service/PathService.java
deleted file mode 100644
index 664d045..0000000
--- a/core/src/main/java/org/apache/carbondata/core/service/PathService.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.service;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-/**
- * Create helper to get path details
- */
-public interface PathService {
-
-  /**
-   * @param absoluteTableIdentifier
-   * @param dictionaryColumnUniqueIdentifier
-   * @return store path related to tables
-   */
-  CarbonTablePath getCarbonTablePath(AbsoluteTableIdentifier absoluteTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier);
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java b/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java
deleted file mode 100644
index cb4ca25..0000000
--- a/core/src/main/java/org/apache/carbondata/core/service/impl/PathFactory.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.service.impl;
-
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.service.PathService;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-/**
- * Create helper to get path details
- */
-public class PathFactory implements PathService {
-
-  private static PathService pathService = new PathFactory();
-
-  /**
-   * @param absoluteTableIdentifier
-   * @param
-   * @return store path related to tables
-   */
-  @Override public CarbonTablePath getCarbonTablePath(
-      AbsoluteTableIdentifier absoluteTableIdentifier,
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    if (null != dictionaryColumnUniqueIdentifier && null != dictionaryColumnUniqueIdentifier
-        .getCarbonTablePath()) {
-      return dictionaryColumnUniqueIdentifier.getCarbonTablePath();
-    }
-    return CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-  }
-
-  public static PathService getInstance() {
-    return pathService;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index 3c991e0..1b45be7 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -47,7 +47,6 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import com.google.gson.Gson;
@@ -60,10 +59,10 @@ public class SegmentStatusManager {
   private static final LogService LOG =
       LogServiceFactory.getLogService(SegmentStatusManager.class.getName());
 
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
+  private AbsoluteTableIdentifier identifier;
 
-  public SegmentStatusManager(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
+  public SegmentStatusManager(AbsoluteTableIdentifier identifier) {
+    this.identifier = identifier;
   }
 
   /**
@@ -72,7 +71,7 @@ public class SegmentStatusManager {
    * @return
    */
   public ICarbonLock getTableStatusLock() {
-    return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier, LockUsage.TABLE_STATUS_LOCK);
+    return CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.TABLE_STATUS_LOCK);
   }
 
   /**
@@ -80,9 +79,7 @@ public class SegmentStatusManager {
    */
   public static long getTableStatusLastModifiedTime(AbsoluteTableIdentifier identifier)
       throws IOException {
-    String tableStatusPath = CarbonStorePath
-        .getCarbonTablePath(identifier.getTablePath(), identifier.getCarbonTableIdentifier())
-        .getTableStatusFilePath();
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
     if (!FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) {
       return 0L;
     } else {
@@ -105,10 +102,7 @@ public class SegmentStatusManager {
     List<Segment> listOfInvalidSegments = new ArrayList<>(10);
     List<Segment> listOfStreamSegments = new ArrayList<>(10);
     List<Segment> listOfInProgressSegments = new ArrayList<>(10);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    String dataPath = carbonTablePath.getTableStatusFilePath();
+    String dataPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
     DataInputStream dataInputStream = null;
 
     // Use GSON to deserialize the load information
@@ -308,9 +302,7 @@ public class SegmentStatusManager {
       if (carbonDeleteSegmentLock.lockWithRetries()) {
         LOG.info("Delete segment lock has been successfully acquired");
 
-        CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-            identifier.getTablePath(), identifier.getCarbonTableIdentifier());
-        String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
+        String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
         LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
         if (!FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) {
           // log error.
@@ -390,9 +382,7 @@ public class SegmentStatusManager {
       if (carbonDeleteSegmentLock.lockWithRetries()) {
         LOG.info("Delete segment lock has been successfully acquired");
 
-        CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-            identifier.getTablePath(), identifier.getCarbonTableIdentifier());
-        String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
+        String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
         LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
 
         if (!FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) {
@@ -720,7 +710,7 @@ public class SegmentStatusManager {
       return false;
     }
     boolean loadInProgress = false;
-    String metaPath = carbonTable.getMetaDataFilepath();
+    String metaPath = carbonTable.getMetadataPath();
     LoadMetadataDetails[] listOfLoadFolderDetailsArray =
               SegmentStatusManager.readLoadMetadata(metaPath);
     if (listOfLoadFolderDetailsArray.length != 0) {
@@ -745,7 +735,7 @@ public class SegmentStatusManager {
       return false;
     }
     boolean loadInProgress = false;
-    String metaPath = carbonTable.getMetaDataFilepath();
+    String metaPath = carbonTable.getMetadataPath();
     LoadMetadataDetails[] listOfLoadFolderDetailsArray =
         SegmentStatusManager.readLoadMetadata(metaPath);
     if (listOfLoadFolderDetailsArray.length != 0) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
index 019a20c..25ce0c8 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
@@ -49,7 +49,6 @@ import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
 import org.apache.carbondata.core.mutate.TupleIdEnum;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import com.google.gson.Gson;
@@ -65,25 +64,21 @@ public class SegmentUpdateStatusManager {
   private static final LogService LOG =
       LogServiceFactory.getLogService(SegmentUpdateStatusManager.class.getName());
 
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
+  private AbsoluteTableIdentifier identifier;
   private LoadMetadataDetails[] segmentDetails;
   private SegmentUpdateDetails[] updateDetails;
-  private CarbonTablePath carbonTablePath;
   private Map<String, SegmentUpdateDetails> blockAndDetailsMap;
   private boolean isPartitionTable;
 
   /**
-   * @param absoluteTableIdentifier
+   * @param identifier
    */
-  public SegmentUpdateStatusManager(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-    carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-        absoluteTableIdentifier.getCarbonTableIdentifier());
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+  public SegmentUpdateStatusManager(AbsoluteTableIdentifier identifier) {
+    this.identifier = identifier;
     // current it is used only for read function scenarios, as file update always requires to work
     // on latest file status.
     segmentDetails =
-        segmentStatusManager.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
+        SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(identifier.getTablePath()));
     if (segmentDetails.length > 0) {
       isPartitionTable = segmentDetails[0].getSegmentFile() != null;
     }
@@ -133,8 +128,6 @@ public class SegmentUpdateStatusManager {
 
   }
 
-
-
   /**
    * Returns the LoadMetadata Details
    * @return
@@ -165,7 +158,7 @@ public class SegmentUpdateStatusManager {
    * @return
    */
   public ICarbonLock getTableUpdateStatusLock() {
-    return CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier,
+    return CarbonLockFactory.getCarbonLockObj(identifier,
         LockUsage.TABLE_UPDATE_STATUS_LOCK);
   }
 
@@ -181,7 +174,8 @@ public class SegmentUpdateStatusManager {
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     String endTimeStamp = "";
     String startTimeStamp = "";
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+    String segmentPath = CarbonTablePath.getSegmentPath(
+        identifier.getTablePath(), segmentId);
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
     for (LoadMetadataDetails eachSeg : segmentDetails) {
@@ -239,6 +233,9 @@ public class SegmentUpdateStatusManager {
    * @throws Exception
    */
   public String[] getDeleteDeltaFilePath(String blockFilePath, String segmentId) throws Exception {
+//    int tableFactPathLength = CarbonTablePath.getFactDir(identifier.getTablePath()).length() + 1;
+//    String blockId = blockFilePath.substring(tableFactPathLength);
+
     String blockId = CarbonUtil.getBlockId(absoluteTableIdentifier, blockFilePath, segmentId);
     String tupleId;
     if (isPartitionTable) {
@@ -260,9 +257,6 @@ public class SegmentUpdateStatusManager {
    */
   public List<String> getDeltaFiles(String tupleId, String extension) throws Exception {
     try {
-      CarbonTablePath carbonTablePath = CarbonStorePath
-          .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-              absoluteTableIdentifier.getCarbonTableIdentifier());
       String segment = CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID);
       String completeBlockName = CarbonTablePath.addDataPartPrefix(
           CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.BLOCK_ID)
@@ -449,16 +443,12 @@ public class SegmentUpdateStatusManager {
       final String fileExtension, final boolean excludeOriginalFact,
       CarbonFile[] allFilesOfSegment, boolean isAbortedFile) {
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
     String endTimeStamp = "";
     String startTimeStamp = "";
     long factTimeStamp = 0;
 
-    LoadMetadataDetails[] segmentDetails =
-        segmentStatusManager.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
+    LoadMetadataDetails[] segmentDetails = SegmentStatusManager.readLoadMetadata(
+        CarbonTablePath.getMetadataPath(identifier.getTablePath()));
 
     for (LoadMetadataDetails eachSeg : segmentDetails) {
       if (eachSeg.getLoadName().equalsIgnoreCase(segmentId)) {
@@ -663,13 +653,9 @@ public class SegmentUpdateStatusManager {
       return new SegmentUpdateDetails[0];
     }
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-
     String tableUpdateStatusPath =
-        carbonTablePath.getMetadataDirectoryPath() + CarbonCommonConstants.FILE_SEPARATOR
-            + tableUpdateStatusIdentifier;
+        CarbonTablePath.getMetadataPath(identifier.getTablePath()) +
+            CarbonCommonConstants.FILE_SEPARATOR + tableUpdateStatusIdentifier;
     AtomicFileOperations fileOperation = new AtomicFileOperationsImpl(tableUpdateStatusPath,
         FileFactory.getFileType(tableUpdateStatusPath));
 
@@ -697,12 +683,9 @@ public class SegmentUpdateStatusManager {
    * @return updateStatusFileName
    */
   private String getUpdatedStatusIdentifier() {
-    SegmentStatusManager ssm = new SegmentStatusManager(absoluteTableIdentifier);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
     LoadMetadataDetails[] loadDetails =
-        ssm.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
+        SegmentStatusManager.readLoadMetadata(
+            CarbonTablePath.getMetadataPath(identifier.getTablePath()));
     if (loadDetails.length == 0) {
       return null;
     }
@@ -717,13 +700,9 @@ public class SegmentUpdateStatusManager {
    */
   public void writeLoadDetailsIntoFile(List<SegmentUpdateDetails> listOfSegmentUpdateDetailsArray,
       String updateStatusFileIdentifier) throws IOException {
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-
     String fileLocation =
-        carbonTablePath.getMetadataDirectoryPath() + CarbonCommonConstants.FILE_SEPARATOR
+        CarbonTablePath.getMetadataPath(identifier.getTablePath())
+            + CarbonCommonConstants.FILE_SEPARATOR
             + CarbonUpdateUtil.getUpdateStatusFileName(updateStatusFileIdentifier);
 
     AtomicFileOperations fileWrite =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 0cc783e..0adc3a5 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -1088,20 +1088,18 @@ public final class CarbonUtil {
    *
    * @param taskId
    * @param tableBlockInfoList
-   * @param absoluteTableIdentifier
+   * @param identifier
    */
   public static long calculateDriverBTreeSize(String taskId, String bucketNumber,
-      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier absoluteTableIdentifier) {
+      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier identifier) {
     // 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.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
     // geting the index file path
     //TODO need to pass proper partition number when partiton will be supported
-    String carbonIndexFilePath = carbonTablePath
-        .getCarbonIndexFilePath(taskId, tableBlockInfoList.get(0).getSegmentId(),
+    String carbonIndexFilePath = CarbonTablePath
+        .getCarbonIndexFilePath(identifier.getTablePath(), taskId,
+            tableBlockInfoList.get(0).getSegmentId(),
             bucketNumber, CarbonTablePath.DataFileUtil
                 .getTimeStampFromFileName(tableBlockInfoList.get(0).getFilePath()),
             tableBlockInfoList.get(0).getVersion());
@@ -1328,23 +1326,21 @@ public final class CarbonUtil {
    *
    * @param taskId                  task id of the file
    * @param tableBlockInfoList      list of table block
-   * @param absoluteTableIdentifier absolute table identifier
+   * @param identifier absolute table identifier
    * @return list of block info
    * @throws IOException if any problem while reading
    */
   public static List<DataFileFooter> readCarbonIndexFile(String taskId, String bucketNumber,
-      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier absoluteTableIdentifier)
+      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier identifier)
       throws IOException {
     // 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.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
     // geting the index file path
     //TODO need to pass proper partition number when partiton will be supported
-    String carbonIndexFilePath = carbonTablePath
-        .getCarbonIndexFilePath(taskId, tableBlockInfoList.get(0).getSegmentId(),
+    String carbonIndexFilePath = CarbonTablePath
+        .getCarbonIndexFilePath(identifier.getTablePath(), taskId,
+            tableBlockInfoList.get(0).getSegmentId(),
             bucketNumber, CarbonTablePath.DataFileUtil
                 .getTimeStampFromFileName(tableBlockInfoList.get(0).getFilePath()),
             tableBlockInfoList.get(0).getVersion());
@@ -2204,21 +2200,6 @@ public final class CarbonUtil {
   }
 
   /**
-   * get the parent folder of old table path and returns the new tablePath by appending new
-   * tableName to the parent
-   *
-   * @param carbonTablePath       Old tablePath
-   * @param newTableName          new table name
-   * @return the new table path
-   */
-  public static String getNewTablePath(
-      Path carbonTablePath,
-      String newTableName) {
-    Path parentPath = carbonTablePath.getParent();
-    return parentPath.toString() + CarbonCommonConstants.FILE_SEPARATOR + newTableName;
-  }
-
-  /**
    * This method will calculate the data size and index size for carbon table
    */
   public static Map<String, Long> calculateDataIndexSize(CarbonTable carbonTable)
@@ -2228,18 +2209,17 @@ public final class CarbonUtil {
     long indexSize = 0L;
     long lastUpdateTime = 0L;
     boolean needUpdate = false;
-    AbsoluteTableIdentifier absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
+    AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
     String isCalculated = CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.ENABLE_CALCULATE_SIZE,
             CarbonCommonConstants.DEFAULT_ENABLE_CALCULATE_SIZE);
     if (isCalculated.equalsIgnoreCase("true")) {
-      SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+      SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
       ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
       try {
         if (carbonLock.lockWithRetries()) {
           LOGGER.info("Acquired lock for table for table status updation");
-          String metadataPath = carbonTable.getMetaDataFilepath();
+          String metadataPath = carbonTable.getMetadataPath();
           LoadMetadataDetails[] loadMetadataDetails =
               SegmentStatusManager.readLoadMetadata(metadataPath);
 
@@ -2253,8 +2233,8 @@ public final class CarbonUtil {
               if (null == dsize || null == isize) {
                 needUpdate = true;
                 LOGGER.info("It is an old segment, need calculate data size and index size again");
-                HashMap<String, Long> map = CarbonUtil
-                    .getDataSizeAndIndexSize(carbonTablePath, loadMetadataDetail.getLoadName());
+                HashMap<String, Long> map = CarbonUtil.getDataSizeAndIndexSize(
+                    identifier.getTablePath(), loadMetadataDetail.getLoadName());
                 dsize = String.valueOf(map.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE));
                 isize = String.valueOf(map.get(CarbonCommonConstants.CARBON_TOTAL_INDEX_SIZE));
                 loadMetadataDetail.setDataSize(dsize);
@@ -2266,10 +2246,12 @@ public final class CarbonUtil {
           }
           // If it contains old segment, write new load details
           if (needUpdate) {
-            SegmentStatusManager.writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePath(),
+            SegmentStatusManager.writeLoadDetailsIntoFile(
+                CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()),
                 loadMetadataDetails);
           }
-          String tableStatusPath = carbonTablePath.getTableStatusFilePath();
+          String tableStatusPath =
+              CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
           if (FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) {
             lastUpdateTime =
                 FileFactory.getCarbonFile(tableStatusPath, FileFactory.getFileType(tableStatusPath))
@@ -2296,12 +2278,12 @@ public final class CarbonUtil {
   }
 
   // Get the total size of carbon data and the total size of carbon index
-  private static HashMap<String, Long> getDataSizeAndIndexSize(CarbonTablePath carbonTablePath,
+  private static HashMap<String, Long> getDataSizeAndIndexSize(String tablePath,
       String segmentId) throws IOException {
     long carbonDataSize = 0L;
     long carbonIndexSize = 0L;
     HashMap<String, Long> dataAndIndexSize = new HashMap<String, Long>();
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+    String segmentPath = CarbonTablePath.getSegmentPath(tablePath, segmentId);
     FileFactory.FileType fileType = FileFactory.getFileType(segmentPath);
     switch (fileType) {
       case HDFS:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java
deleted file mode 100644
index b864e6e..0000000
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonSharedDictionaryPath.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.util.path;
-
-import java.io.File;
-
-/**
- * Helps to get Shared dimension files path.
- */
-public class CarbonSharedDictionaryPath {
-
-  private static final String SHAREDDIM_DIR = "SharedDictionary";
-  private static final String DICTIONARY_EXT = ".dict";
-  private static final String DICTIONARY_META_EXT = ".dictmeta";
-  private static final String SORT_INDEX_EXT = ".sortindex";
-
-  /***
-   * @param storePath    store path
-   * @param databaseName data base name
-   * @param columnId     unique column identifier
-   * @return absolute path of shared dictionary file
-   */
-  public static String getDictionaryFilePath(String storePath, String databaseName,
-      String columnId) {
-    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
-        + DICTIONARY_EXT;
-  }
-
-  /***
-   * @param storePath    store path
-   * @param databaseName data base name
-   * @param columnId     unique column identifier
-   * @return absolute path of shared dictionary meta file
-   */
-  public static String getDictionaryMetaFilePath(String storePath, String databaseName,
-      String columnId) {
-    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
-        + DICTIONARY_META_EXT;
-  }
-
-  /***
-   * @param storePath    store path
-   * @param databaseName data base name
-   * @param columnId     unique column identifier
-   * @return absolute path of shared dictionary sort index file
-   */
-  public static String getSortIndexFilePath(String storePath, String databaseName,
-      String columnId) {
-    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
-        + SORT_INDEX_EXT;
-  }
-
-  private static String getSharedDictionaryDir(String storePath, String databaseName) {
-    return storePath + File.separator + databaseName + File.separator + SHAREDDIM_DIR;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java
deleted file mode 100644
index e57448d..0000000
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonStorePath.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.util.path;
-
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-
-import org.apache.hadoop.fs.Path;
-
-/**
- * Helps to get Store content paths.
- */
-public class CarbonStorePath extends Path {
-
-  private String storePath;
-
-  public CarbonStorePath(String storePathString) {
-    super(storePathString);
-    this.storePath = storePathString;
-  }
-
-  /**
-   * gets CarbonTablePath object to manage table paths
-   *
-   * @param tablePath the store path of the segment
-   * @param tableIdentifier identifier of carbon table that the segment belong to
-   * @return the store location of the segment
-   */
-  public static CarbonTablePath getCarbonTablePath(String tablePath,
-      CarbonTableIdentifier tableIdentifier) {
-    return new CarbonTablePath(tableIdentifier, tablePath);
-  }
-
-  public static CarbonTablePath getCarbonTablePath(AbsoluteTableIdentifier identifier) {
-    CarbonTableIdentifier id = identifier.getCarbonTableIdentifier();
-    return new CarbonTablePath(id, identifier.getTablePath());
-  }
-
-  /**
-   * gets CarbonTablePath object to manage table paths
-   */
-  public CarbonTablePath getCarbonTablePath(CarbonTableIdentifier tableIdentifier) {
-    return CarbonStorePath.getCarbonTablePath(storePath, tableIdentifier);
-  }
-
-  @Override public boolean equals(Object o) {
-    if (!(o instanceof CarbonStorePath)) {
-      return false;
-    }
-    CarbonStorePath path = (CarbonStorePath)o;
-    return storePath.equals(path.storePath) && super.equals(o);
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode() + storePath.hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index 293257b..0164151 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -22,16 +22,14 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 
 import org.apache.hadoop.fs.Path;
 
-
 /**
  * Helps to get Table content paths.
  */
-public class CarbonTablePath extends Path {
+public class CarbonTablePath {
 
   private static final String METADATA_DIR = "Metadata";
   private static final String DICTIONARY_EXT = ".dict";
@@ -54,19 +52,10 @@ public class CarbonTablePath extends Path {
   private static final String STREAMING_LOG_DIR = "log";
   private static final String STREAMING_CHECKPOINT_DIR = "checkpoint";
 
-  private String tablePath;
-  private CarbonTableIdentifier carbonTableIdentifier;
-
   /**
-   * structure CarbonTablePath object to manage table paths
-   *
-   * @param carbonTableIdentifier identifier of carbon table that the segment belong to
-   * @param tablePathString the store path of the segment
+   * This class provides static utility only.
    */
-  public CarbonTablePath(CarbonTableIdentifier carbonTableIdentifier, String tablePathString) {
-    super(tablePathString);
-    this.carbonTableIdentifier = carbonTableIdentifier;
-    this.tablePath = tablePathString;
+  private CarbonTablePath() {
   }
 
   /**
@@ -125,107 +114,63 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * gets table path
-   */
-  public String getPath() {
-    return tablePath;
-  }
-
-  /**
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of dictionary file
    */
-  public String getDictionaryFilePath(String columnId) {
-    return getMetaDataDir() + File.separator + getDictionaryFileName(columnId);
+  public static String getDictionaryFilePath(String tablePath, String columnId) {
+    return getMetadataPath(tablePath) + File.separator + getDictionaryFileName(columnId);
   }
 
   /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of dictionary file
    */
-  public String getDictionaryFilePath(String dictionaryPath, String columnId) {
+  public static String getExternalDictionaryFilePath(String dictionaryPath, String columnId) {
     return dictionaryPath + File.separator + getDictionaryFileName(columnId);
   }
 
   /**
-   * This method will return the metadata directory location for a table
-   *
-   * @return
-   */
-  public String getMetadataDirectoryPath() {
-    return getMetaDataDir();
-  }
-
-  /**
-   * Return metadata path based on `tablePath`
+   * Return metadata path
    */
   public static String getMetadataPath(String tablePath) {
     return tablePath + File.separator + METADATA_DIR;
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary meta file
+   * Return absolute path of dictionary meta file
    */
-  public String getDictionaryMetaFilePath(String columnId) {
-    return getMetaDataDir() + File.separator + columnId + DICTIONARY_META_EXT;
-  }
-
-  /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
-   */
-  public String getDictionaryMetaFilePath(String dictionaryPath, String columnId) {
+  public static String getExternalDictionaryMetaFilePath(String dictionaryPath, String columnId) {
     return dictionaryPath + File.separator + columnId + DICTIONARY_META_EXT;
   }
 
   /**
-   * @param columnId unique column identifier
-   * @return absolute path of sort index file
+   * Return absolute path of dictionary meta file
    */
-  public String getSortIndexFilePath(String columnId) {
-    return getMetaDataDir() + File.separator + columnId + SORT_INDEX_EXT;
+  public static String getDictionaryMetaFilePath(String tablePath, String columnId) {
+    return getMetadataPath(tablePath) + File.separator + columnId + DICTIONARY_META_EXT;
   }
 
   /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @return absolute path of dictionary file
+   * Return absolute path of sort index file
    */
-  public String getSortIndexFilePath(String dictionaryPath, String columnId) {
-    return dictionaryPath + File.separator + columnId + SORT_INDEX_EXT;
+  public static String getSortIndexFilePath(String tablePath, String columnId) {
+    return getMetadataPath(tablePath) + File.separator + columnId + SORT_INDEX_EXT;
   }
 
   /**
-   *
-   * @param columnId
-   * @param dictOffset
-   * @return absolute path of sortindex with appeneded dictionary offset
+   * Return sortindex file path based on specified dictionary path
    */
-  public String getSortIndexFilePath(String columnId, long dictOffset) {
-    return getMetaDataDir() + File.separator + columnId + "_" + dictOffset + SORT_INDEX_EXT;
+  public static String getExternalSortIndexFilePath(String dictionaryPath, String columnId) {
+    return dictionaryPath + File.separator + columnId + SORT_INDEX_EXT;
   }
 
   /**
-   * @param dictionaryPath
-   * @param columnId unique column identifier
-   * @param dictOffset
-   * @return absolute path of dictionary file
+   * Return sortindex file path for columnId and offset based on specified dictionary path
    */
-  public String getSortIndexFilePath(String dictionaryPath, String columnId, long dictOffset) {
+  public static String getExternalSortIndexFilePath(String dictionaryPath, String columnId,
+      long dictOffset) {
     return dictionaryPath + File.separator + columnId + "_" + dictOffset + SORT_INDEX_EXT;
   }
 
   /**
-   * @return absolute path of schema file
-   */
-  public String getSchemaFilePath() {
-    return getActualSchemaFilePath(tablePath);
-  }
-
-  /**
    * return the schema file path
    * @param tablePath path to table files
    * @return schema file path
@@ -250,10 +195,10 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * @return absolute path of table status file
+   * Return absolute path of table status file
    */
-  public String getTableStatusFilePath() {
-    return getMetaDataDir() + File.separator + TABLE_STATUS_FILE;
+  public static String getTableStatusFilePath(String tablePath) {
+    return getMetadataPath(tablePath) + File.separator + TABLE_STATUS_FILE;
   }
 
   public String getTableStatusFilePathWithUUID(String uuid) {
@@ -272,9 +217,9 @@ public class CarbonTablePath extends Path {
    * @param factUpdateTimeStamp unique identifier to identify an update
    * @return absolute path of data file stored in carbon data format
    */
-  public String getCarbonDataFilePath(String segmentId, Integer filePartNo, Long taskNo,
-      int batchNo, int bucketNumber, String factUpdateTimeStamp) {
-    return getSegmentDir(segmentId) + File.separator + getCarbonDataFileName(
+  public static String getCarbonDataFilePath(String tablePath, String segmentId, Integer filePartNo,
+      Long taskNo, int batchNo, int bucketNumber, String factUpdateTimeStamp) {
+    return getSegmentPath(tablePath, segmentId) + File.separator + getCarbonDataFileName(
         filePartNo, taskNo, bucketNumber, batchNo, factUpdateTimeStamp);
   }
 
@@ -286,9 +231,9 @@ public class CarbonTablePath extends Path {
    * @param segmentId   segment number
    * @return full qualified carbon index path
    */
-  public String getCarbonIndexFilePath(final String taskId, final String segmentId,
-      final String bucketNumber) {
-    String segmentDir = getSegmentDir(segmentId);
+  private static String getCarbonIndexFilePath(final String tablePath, final String taskId,
+      final String segmentId, final String bucketNumber) {
+    String segmentDir = getSegmentPath(tablePath, segmentId);
     CarbonFile carbonFile =
         FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir));
 
@@ -321,27 +266,28 @@ public class CarbonTablePath extends Path {
    *        timestamp
    * @return carbon index file path
    */
-  public String getCarbonIndexFilePath(String taskId, String segmentId, String bucketNumber,
-      String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
+  public static String getCarbonIndexFilePath(String tablePath, String taskId, String segmentId,
+      String bucketNumber, String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
     switch (columnarFormatVersion) {
       case V1:
       case V2:
-        return getCarbonIndexFilePath(taskId, segmentId, bucketNumber);
+        return getCarbonIndexFilePath(tablePath, taskId, segmentId, bucketNumber);
       default:
-        String segmentDir = getSegmentDir(segmentId);
+        String segmentDir = getSegmentPath(tablePath, segmentId);
         return segmentDir + File.separator + getCarbonIndexFileName(taskId,
             Integer.parseInt(bucketNumber), timeStamp);
     }
   }
 
-  public String getCarbonIndexFilePath(String taskId, String segmentId, int batchNo,
-      String bucketNumber, String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
+  public static String getCarbonIndexFilePath(String tablePath, String taskId, String segmentId,
+      int batchNo, String bucketNumber, String timeStamp,
+      ColumnarFormatVersion columnarFormatVersion) {
     switch (columnarFormatVersion) {
       case V1:
       case V2:
-        return getCarbonIndexFilePath(taskId, segmentId, bucketNumber);
+        return getCarbonIndexFilePath(tablePath, taskId, segmentId, bucketNumber);
       default:
-        String segmentDir = getSegmentDir(segmentId);
+        String segmentDir = getSegmentPath(tablePath, segmentId);
         return segmentDir + File.separator + getCarbonIndexFileName(Long.parseLong(taskId),
             Integer.parseInt(bucketNumber), batchNo, timeStamp);
     }
@@ -356,13 +302,10 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * Gets absolute path of data file
-   *
-   * @param segmentId   unique partition identifier
-   * @return absolute path of data file stored in carbon data format
+   * Return the segment path from table path and segmentid
    */
-  public String getCarbonDataDirectoryPath(String segmentId) {
-    return getSegmentDir(segmentId);
+  public static String getSegmentPath(String tablePath, String segmentId) {
+    return getPartitionDir(tablePath) + File.separator + SEGMENT_PREFIX + segmentId;
   }
 
   /**
@@ -400,48 +343,46 @@ public class CarbonTablePath extends Path {
     return segmentDir + File.separator + getCarbonStreamIndexFileName();
   }
 
-  public String getSegmentDir(String segmentId) {
-    return getPartitionDir() + File.separator + SEGMENT_PREFIX + segmentId;
-  }
-
   // This partition is not used in any code logic, just keep backward compatibility
   public static final String DEPRECATED_PATITION_ID = "0";
 
-  public String getPartitionDir() {
-    return getFactDir() + File.separator + PARTITION_PREFIX +
-        CarbonTablePath.DEPRECATED_PATITION_ID;
+  /**
+   * Return true if tablePath exists
+   */
+  public static boolean exists(String tablePath) {
+    return FileFactory.getCarbonFile(tablePath, FileFactory.getFileType(tablePath)).exists();
   }
 
-  private String getMetaDataDir() {
-    return tablePath + File.separator + METADATA_DIR;
+  public static String getPartitionDir(String tablePath) {
+    return getFactDir(tablePath) + File.separator + PARTITION_PREFIX +
+        CarbonTablePath.DEPRECATED_PATITION_ID;
   }
 
-  public String getFactDir() {
+  public static String getFactDir(String tablePath) {
     return tablePath + File.separator + FACT_DIR;
   }
 
-  public String getStreamingLogDir() {
+  public static String getStreamingLogDir(String tablePath) {
     return tablePath + File.separator + STREAMING_DIR + File.separator + STREAMING_LOG_DIR;
   }
 
-  public String getStreamingCheckpointDir() {
+  public static String getStreamingCheckpointDir(String tablePath) {
     return tablePath + File.separator + STREAMING_DIR + File.separator + STREAMING_CHECKPOINT_DIR;
   }
 
-  public CarbonTableIdentifier getCarbonTableIdentifier() {
-    return carbonTableIdentifier;
-  }
-
-  @Override public boolean equals(Object o) {
-    if (!(o instanceof CarbonTablePath)) {
-      return false;
-    }
-    CarbonTablePath path = (CarbonTablePath) o;
-    return tablePath.equals(path.tablePath) && super.equals(o);
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode() + tablePath.hashCode();
+  /**
+   * get the parent folder of old table path and returns the new tablePath by appending new
+   * tableName to the parent
+   *
+   * @param tablePath         Old tablePath
+   * @param newTableName      new table name
+   * @return the new table path
+   */
+  public static String getNewTablePath(
+      String tablePath,
+      String newTableName) {
+    Path parentPath = new Path(tablePath).getParent();
+    return parentPath.toString() + CarbonCommonConstants.FILE_SEPARATOR + newTableName;
   }
 
   /**
@@ -460,11 +401,8 @@ public class CarbonTablePath extends Path {
       return fileName.substring(startIndex, endIndex);
     }
 
-
     /**
-     * This will return the timestamp present in the delete delta file.
-     * @param fileName
-     * @return
+     * Return the timestamp present in the delete delta file.
      */
     public static String getTimeStampFromDeleteDeltaFile(String fileName) {
       return fileName.substring(fileName.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1,
@@ -472,9 +410,7 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * This will return the timestamp present in the delete delta file.
-     * @param fileName
-     * @return
+     * Return the timestamp present in the delete delta file.
      */
     public static String getBlockNameFromDeleteDeltaFile(String fileName) {
       return fileName.substring(0,
@@ -482,7 +418,7 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * gets updated timestamp information from given carbon data file name
+     * Return the updated timestamp information from given carbon data file name
      */
     public static String getBucketNo(String carbonFilePath) {
       // Get the file name from path
@@ -500,7 +436,7 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * gets file part number information from given carbon data file name
+     * Return the file part number information from given carbon data file name
      */
     public static String getPartNo(String carbonDataFileName) {
       // Get the file name from path
@@ -512,7 +448,7 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * gets updated timestamp information from given carbon data file name
+     * Return the updated timestamp information from given carbon data file name
      */
     public static String getTaskNo(String carbonDataFileName) {
       // Get the file name from path
@@ -525,35 +461,30 @@ public class CarbonTablePath extends Path {
     }
 
     /**
-     * get the taskId part from taskNo(include taskId + batchNo)
-     * @param taskNo
-     * @return
+     * Return the taskId part from taskNo(include taskId + batchNo)
      */
     public static long getTaskIdFromTaskNo(String taskNo) {
       return Long.parseLong(taskNo.split(BATCH_PREFIX)[0]);
     }
 
+    /**
+     * Return the batch number from taskNo string
+     */
     public static int getBatchNoFromTaskNo(String taskNo) {
       return Integer.parseInt(taskNo.split(BATCH_PREFIX)[1]);
     }
 
     /**
-     * Gets the file name from file path
+     * Return the file name from file path
      */
-    private static String getFileName(String carbonDataFileName) {
-      int endIndex = carbonDataFileName.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR);
+    private static String getFileName(String dataFilePath) {
+      int endIndex = dataFilePath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR);
       if (endIndex > -1) {
-        return carbonDataFileName.substring(endIndex + 1, carbonDataFileName.length());
+        return dataFilePath.substring(endIndex + 1, dataFilePath.length());
       } else {
-        return carbonDataFileName;
+        return dataFilePath;
       }
     }
-  }
-
-  /**
-   * To manage data path and composition
-   */
-  public static class DataPathUtil {
 
     /**
      * gets segement id from given absolute data file path
@@ -561,11 +492,11 @@ public class CarbonTablePath extends Path {
     public static String getSegmentId(String dataFileAbsolutePath) {
       // find segment id from last of data file path
       String tempdataFileAbsolutePath = dataFileAbsolutePath.replace(
-              CarbonCommonConstants.WINDOWS_FILE_SEPARATOR, CarbonCommonConstants.FILE_SEPARATOR);
+          CarbonCommonConstants.WINDOWS_FILE_SEPARATOR, CarbonCommonConstants.FILE_SEPARATOR);
       int endIndex = tempdataFileAbsolutePath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR);
       // + 1 for size of "/"
       int startIndex = tempdataFileAbsolutePath.lastIndexOf(
-              CarbonCommonConstants.FILE_SEPARATOR, endIndex - 1) + 1;
+          CarbonCommonConstants.FILE_SEPARATOR, endIndex - 1) + 1;
       String segmentDirStr = dataFileAbsolutePath.substring(startIndex, endIndex);
       //identify id in segment_<id>
       String[] segmentDirSplits = segmentDirStr.split("_");
@@ -597,19 +528,16 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * returns the carbondata file name
-   *
-   * @param carbonDataFilePath carbondata file path
-   * @return
+   * Return the carbondata file name
    */
   public static String getCarbonDataFileName(String carbonDataFilePath) {
-    return carbonDataFilePath
-        .substring(carbonDataFilePath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR) + 1,
-            carbonDataFilePath.indexOf(CARBON_DATA_EXT));
+    return carbonDataFilePath.substring(
+        carbonDataFilePath.lastIndexOf(CarbonCommonConstants.FILE_SEPARATOR) + 1,
+        carbonDataFilePath.indexOf(CARBON_DATA_EXT));
   }
 
   /**
-   * @return prefix of carbon data
+   * Return prefix of carbon data
    */
   public static String getCarbonDataPrefix() {
     return DATA_PART_PREFIX;
@@ -665,40 +593,6 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * This method will append strings in path and return block id
-   *
-   * @param shortBlockId
-   * @return blockId
-   */
-  public static String getBlockId(String shortBlockId) {
-    String[] splitRecords = shortBlockId.split(CarbonCommonConstants.FILE_SEPARATOR);
-    StringBuffer sb = new StringBuffer();
-    for (int i = 0; i < splitRecords.length; i++) {
-      if (i == 0) {
-        sb.append(PARTITION_PREFIX);
-        sb.append(splitRecords[i]);
-      } else if (i == 1) {
-        sb.append(CarbonCommonConstants.FILE_SEPARATOR);
-        sb.append(SEGMENT_PREFIX);
-        sb.append(splitRecords[i]);
-      } else if (i == 2) {
-        sb.append(CarbonCommonConstants.FILE_SEPARATOR);
-        sb.append(DATA_PART_PREFIX);
-        sb.append(splitRecords[i]);
-      } else if (i == 3) {
-        sb.append(CarbonCommonConstants.FILE_SEPARATOR);
-        sb.append(splitRecords[i]);
-        sb.append(CARBON_DATA_EXT);
-      } else {
-        sb.append(CarbonCommonConstants.FILE_SEPARATOR);
-        sb.append(splitRecords[i]);
-      }
-    }
-    return sb.toString();
-  }
-
-
-  /**
    * adds data part prefix to given value
    * @return partition prefix
    */
@@ -728,13 +622,6 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * Get the segment path from table path and segmentid
-   */
-  public static String getSegmentPath(String tablePath, String segmentId) {
-    return tablePath + "/Fact/Part0/Segment_" + segmentId;
-  }
-
-  /**
    * Get the segment file locations of table
    */
   public static String getSegmentFilesLocation(String tablePath) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
index 31e44a2..7d829b9 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
@@ -34,7 +34,6 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -48,7 +47,7 @@ public class AbstractDictionaryCacheTest {
 
   protected CarbonTableIdentifier carbonTableIdentifier;
 
-  protected AbsoluteTableIdentifier absoluteTableIdentifier;
+  protected AbsoluteTableIdentifier identifier;
 
   protected String databaseName;
 
@@ -107,7 +106,7 @@ public class AbstractDictionaryCacheTest {
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
       String columnId) {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.STRING);
-    return new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+    return new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
         DataTypes.STRING);
   }
 
@@ -130,13 +129,11 @@ public class AbstractDictionaryCacheTest {
       throws IOException {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
             columnIdentifier.getDataType());
     CarbonDictionaryWriter carbonDictionaryWriter =
         new CarbonDictionaryWriterImpl(dictionaryColumnUniqueIdentifier);
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    CarbonUtil.checkAndCreateFolder(carbonTablePath.getMetadataDirectoryPath());
+    CarbonUtil.checkAndCreateFolder(CarbonTablePath.getMetadataPath(identifier.getTablePath()));
     List<byte[]> valueList = convertStringListToByteArray(data);
     try {
       carbonDictionaryWriter.write(valueList);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
index d0aedd4..c36c89d 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
@@ -56,7 +56,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
     carbonTableIdentifier =
         new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    absoluteTableIdentifier =
+    identifier =
         AbsoluteTableIdentifier.from(carbonStorePath + "/" + databaseName + "/" + tableName,
             carbonTableIdentifier);
     columnIdentifiers = new String[] { "name", "place" };
@@ -67,7 +67,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
 
   @After public void tearDown() throws Exception {
     carbonTableIdentifier = null;
-    absoluteTableIdentifier = null;
+    identifier = null;
     forwardDictionaryCache = null;
     deleteStorePath();
   }
@@ -217,7 +217,7 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
   private void writeSortIndexFile(List<String> data, String columnId) throws IOException {
 	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier,
+        new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier,
             columnIdentifier.getDataType());
     Map<String, Integer> dataToSurrogateKeyMap = new HashMap<>(data.size());
     int surrogateKey = 0;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
index 01cb3a9..d2bf2e3 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
@@ -58,7 +58,7 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
     carbonTableIdentifier =
         new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
-    absoluteTableIdentifier = AbsoluteTableIdentifier.from(
+    identifier = AbsoluteTableIdentifier.from(
         carbonStorePath + "/" + databaseName + "/" + tableName, carbonTableIdentifier);
     columnIdentifiers = new String[] { "name", "place" };
     deleteStorePath();
@@ -69,7 +69,7 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
   @After public void tearDown() throws Exception {
     carbonTableIdentifier = null;
     reverseDictionaryCache = null;
-    absoluteTableIdentifier = null;
+    identifier = null;
     deleteStorePath();
   }
 
@@ -276,6 +276,6 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
 	      String columnId) {
 	    ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataTypes.DOUBLE);
-    return new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier, columnIdentifier);
+    return new DictionaryColumnUniqueIdentifier(identifier, columnIdentifier);
 	  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
index d3c3bc3..ecabfd4 100644
--- a/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImplTest.java
@@ -28,8 +28,6 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.service.CarbonCommonFactory;
-import org.apache.carbondata.core.service.PathService;
-import org.apache.carbondata.core.service.impl.PathFactory;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -58,12 +56,6 @@ public class CarbonDictionaryReaderImplTest {
   }
 
   @Test public void testRead() throws Exception {
-    new MockUp<CarbonCommonFactory>() {
-      @Mock public PathService getPathService() {
-
-        return new PathFactory();
-      }
-    };
     new MockUp<CarbonDictionaryMetadataReaderImpl>() {
       @Mock public List<CarbonDictionaryColumnMetaChunk> read() throws IOException {
         CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunks =


[12/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
index de97e82..540607d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -34,8 +34,8 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
@@ -58,7 +58,7 @@ import org.apache.carbondata.core.scan.filter.intf.RowImpl;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -68,20 +68,20 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(RowLevelFilterExecuterImpl.class.getName());
-  protected List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
-  protected List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
+  List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
+  List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
   protected Expression exp;
   protected AbsoluteTableIdentifier tableIdentifier;
   protected SegmentProperties segmentProperties;
   /**
    * it has index at which given dimension is stored in file
    */
-  protected int[] dimensionBlocksIndex;
+  int[] dimensionChunkIndex;
 
   /**
    * it has index at which given measure is stored in file
    */
-  protected int[] measureBlocksIndex;
+  int[] measureChunkIndex;
 
   private Map<Integer, GenericQueryType> complexDimensionInfoMap;
 
@@ -89,18 +89,18 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * flag to check whether the filter dimension is present in current block list of dimensions.
    * Applicable for restructure scenarios
    */
-  protected boolean[] isDimensionPresentInCurrentBlock;
+  boolean[] isDimensionPresentInCurrentBlock;
 
   /**
    * flag to check whether the filter measure is present in current block list of measures.
    * Applicable for restructure scenarios
    */
-  protected boolean[] isMeasurePresentInCurrentBlock;
+  boolean[] isMeasurePresentInCurrentBlock;
 
   /**
    * is dimension column data is natural sorted
    */
-  protected boolean isNaturalSorted;
+  boolean isNaturalSorted;
 
   /**
    * date direct dictionary generator
@@ -124,10 +124,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     }
     if (this.dimColEvaluatorInfoList.size() > 0) {
       this.isDimensionPresentInCurrentBlock = new boolean[dimColEvaluatorInfoList.size()];
-      this.dimensionBlocksIndex = new int[dimColEvaluatorInfoList.size()];
+      this.dimensionChunkIndex = new int[dimColEvaluatorInfoList.size()];
     } else {
       this.isDimensionPresentInCurrentBlock = new boolean[]{false};
-      this.dimensionBlocksIndex = new int[]{0};
+      this.dimensionChunkIndex = new int[]{0};
     }
     if (null == msrColEvalutorInfoList) {
       this.msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(20);
@@ -136,10 +136,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     }
     if (this.msrColEvalutorInfoList.size() > 0) {
       this.isMeasurePresentInCurrentBlock = new boolean[msrColEvalutorInfoList.size()];
-      this.measureBlocksIndex = new int[msrColEvalutorInfoList.size()];
+      this.measureChunkIndex = new int[msrColEvalutorInfoList.size()];
     } else {
       this.isMeasurePresentInCurrentBlock = new boolean[]{false};
-      this.measureBlocksIndex = new int[] {0};
+      this.measureChunkIndex = new int[] {0};
     }
     this.exp = exp;
     this.tableIdentifier = tableIdentifier;
@@ -148,22 +148,22 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
         DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.DATE);
     this.timestampDictionaryGenerator =
         DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataTypes.TIMESTAMP);
-    initDimensionBlockIndexes();
-    initMeasureBlockIndexes();
+    initDimensionChunkIndexes();
+    initMeasureChunkIndexes();
   }
 
   /**
    * This method will initialize the dimension info for the current block to be
    * used for filtering the data
    */
-  private void initDimensionBlockIndexes() {
+  private void initDimensionChunkIndexes() {
     for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
       // find the dimension in the current block dimensions list
       CarbonDimension dimensionFromCurrentBlock = segmentProperties
           .getDimensionFromCurrentBlock(dimColEvaluatorInfoList.get(i).getDimension());
       if (null != dimensionFromCurrentBlock) {
         dimColEvaluatorInfoList.get(i).setColumnIndex(dimensionFromCurrentBlock.getOrdinal());
-        this.dimensionBlocksIndex[i] = segmentProperties.getDimensionOrdinalToBlockMapping()
+        this.dimensionChunkIndex[i] = segmentProperties.getDimensionOrdinalToChunkMapping()
             .get(dimensionFromCurrentBlock.getOrdinal());
         isDimensionPresentInCurrentBlock[i] = true;
       }
@@ -174,14 +174,14 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * This method will initialize the measure info for the current block to be
    * used for filtering the data
    */
-  private void initMeasureBlockIndexes() {
+  private void initMeasureChunkIndexes() {
     for (int i = 0; i < msrColEvalutorInfoList.size(); i++) {
       // find the measure in the current block measures list
       CarbonMeasure measureFromCurrentBlock = segmentProperties.getMeasureFromCurrentBlock(
           msrColEvalutorInfoList.get(i).getCarbonColumn().getColumnId());
       if (null != measureFromCurrentBlock) {
         msrColEvalutorInfoList.get(i).setColumnIndex(measureFromCurrentBlock.getOrdinal());
-        this.measureBlocksIndex[i] = segmentProperties.getMeasuresOrdinalToBlockMapping()
+        this.measureChunkIndex[i] = segmentProperties.getMeasuresOrdinalToChunkMapping()
             .get(measureFromCurrentBlock.getOrdinal());
         isMeasurePresentInCurrentBlock[i] = true;
       }
@@ -189,9 +189,9 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    readBlocks(blockChunkHolder);
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    readColumnChunks(rawBlockletColumnChunks);
     // CHECKSTYLE:ON
 
     int[] numberOfRows = null;
@@ -199,31 +199,31 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
 
     if (dimColEvaluatorInfoList.size() > 0) {
       if (isDimensionPresentInCurrentBlock[0]) {
-        pageNumbers =
-            blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[0]].getPagesCount();
-        numberOfRows =
-            blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[0]].getRowCount();
+        pageNumbers = rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[0]]
+            .getPagesCount();
+        numberOfRows = rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[0]]
+            .getRowCount();
       } else {
         // specific for restructure case where default values need to be filled
-        pageNumbers = blockChunkHolder.getDataBlock().numberOfPages();
+        pageNumbers = rawBlockletColumnChunks.getDataBlock().numberOfPages();
         numberOfRows = new int[pageNumbers];
         for (int i = 0; i < pageNumbers; i++) {
-          numberOfRows[i] = blockChunkHolder.getDataBlock().getPageRowCount(i);
+          numberOfRows[i] = rawBlockletColumnChunks.getDataBlock().getPageRowCount(i);
         }
       }
     }
     if (msrColEvalutorInfoList.size() > 0) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        pageNumbers =
-            blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]].getPagesCount();
-        numberOfRows =
-            blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]].getRowCount();
+        pageNumbers = rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]]
+            .getPagesCount();
+        numberOfRows = rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]]
+            .getRowCount();
       } else {
         // specific for restructure case where default values need to be filled
-        pageNumbers = blockChunkHolder.getDataBlock().numberOfPages();
+        pageNumbers = rawBlockletColumnChunks.getDataBlock().numberOfPages();
         numberOfRows = new int[pageNumbers];
         for (int i = 0; i < pageNumbers; i++) {
-          numberOfRows[i] = blockChunkHolder.getDataBlock().getPageRowCount(i);
+          numberOfRows[i] = rawBlockletColumnChunks.getDataBlock().getPageRowCount(i);
         }
       }
     }
@@ -234,10 +234,12 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
       BitSet prvBitset = null;
       // if bitset pipe line is enabled then use rowid from previous bitset
       // otherwise use older flow
-      if (!useBitsetPipeLine || null == blockChunkHolder.getBitSetGroup() || null == bitSetGroup
-          .getBitSet(i) || blockChunkHolder.getBitSetGroup().getBitSet(i).isEmpty()) {
+      if (!useBitsetPipeLine ||
+          null == rawBlockletColumnChunks.getBitSetGroup() ||
+          null == bitSetGroup.getBitSet(i) ||
+          rawBlockletColumnChunks.getBitSetGroup().getBitSet(i).isEmpty()) {
         for (int index = 0; index < numberOfRows[i]; index++) {
-          createRow(blockChunkHolder, row, i, index);
+          createRow(rawBlockletColumnChunks, row, i, index);
           Boolean rslt = false;
           try {
             rslt = exp.evaluate(row).getBoolean();
@@ -253,10 +255,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
           }
         }
       } else {
-        prvBitset = blockChunkHolder.getBitSetGroup().getBitSet(i);
+        prvBitset = rawBlockletColumnChunks.getBitSetGroup().getBitSet(i);
         for (int index = prvBitset.nextSetBit(0);
              index >= 0; index = prvBitset.nextSetBit(index + 1)) {
-          createRow(blockChunkHolder, row, i, index);
+          createRow(rawBlockletColumnChunks, row, i, index);
           Boolean rslt = false;
           try {
             rslt = exp.evaluate(row).getBoolean();
@@ -371,8 +373,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * @param index
    * @throws IOException
    */
-  private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int pageIndex, int index)
-      throws IOException {
+  private void createRow(RawBlockletColumnChunks blockChunkHolder, RowIntf row, int pageIndex,
+      int index) throws IOException {
     Object[] record = new Object[dimColEvaluatorInfoList.size() + msrColEvalutorInfoList.size()];
     String memberString;
     for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
@@ -389,14 +391,14 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
           record[dimColumnEvaluatorInfo.getRowIndex()] =
               dimColumnEvaluatorInfo.getDimension().getDefaultValue();
         }
-        DimensionColumnDataChunk columnDataChunk =
-            blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]]
-                .convertToDimColDataChunk(pageIndex);
+        DimensionColumnPage columnDataChunk =
+            blockChunkHolder.getDimensionRawColumnChunks()[dimensionChunkIndex[i]]
+                .decodeColumnPage(pageIndex);
         if (!dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
-            && columnDataChunk instanceof VariableLengthDimensionDataChunk) {
+            && columnDataChunk instanceof VariableLengthDimensionColumnPage) {
 
-          VariableLengthDimensionDataChunk dimensionColumnDataChunk =
-              (VariableLengthDimensionDataChunk) columnDataChunk;
+          VariableLengthDimensionColumnPage dimensionColumnDataChunk =
+              (VariableLengthDimensionColumnPage) columnDataChunk;
           byte[] memberBytes = dimensionColumnDataChunk.getChunkData(index);
           if (null != memberBytes) {
             if (Arrays.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, memberBytes)) {
@@ -409,8 +411,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
                     dimColumnEvaluatorInfo.getDimension().getDataType());
           }
         } else {
-          int dictionaryValue = readSurrogatesFromColumnBlock(blockChunkHolder, index, pageIndex,
-              dimColumnEvaluatorInfo, dimensionBlocksIndex[i]);
+          int dictionaryValue = readSurrogatesFromColumnChunk(blockChunkHolder, index, pageIndex,
+              dimColumnEvaluatorInfo, dimensionChunkIndex[i]);
           if (dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
               && !dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
             memberString =
@@ -428,11 +430,11 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
         }
       } else {
         try {
-          GenericQueryType complexType = complexDimensionInfoMap.get(dimensionBlocksIndex[i]);
+          GenericQueryType complexType = complexDimensionInfoMap.get(dimensionChunkIndex[i]);
           ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
           DataOutputStream dataOutputStream = new DataOutputStream(byteStream);
           complexType.parseBlocksAndReturnComplexColumnByteArray(
-              blockChunkHolder.getDimensionRawDataChunk(), index, pageIndex, dataOutputStream);
+              blockChunkHolder.getDimensionRawColumnChunks(), index, pageIndex, dataOutputStream);
           record[dimColumnEvaluatorInfo.getRowIndex()] = complexType
               .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(byteStream.toByteArray()));
           byteStream.close();
@@ -471,8 +473,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
 
       Object msrValue;
       ColumnPage columnPage =
-          blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]]
-              .convertToColumnPage(pageIndex);
+          blockChunkHolder.getMeasureRawColumnChunks()[measureChunkIndex[0]]
+              .decodeColumnPage(pageIndex);
       if (msrType == DataTypes.BOOLEAN) {
         msrValue = columnPage.getBoolean(index);
       } else if (msrType == DataTypes.SHORT) {
@@ -574,10 +576,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * @param dimColumnEvaluatorInfo
    * @return
    */
-  private int readSurrogatesFromColumnBlock(BlocksChunkHolder blockChunkHolder, int index, int page,
-      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int blockIndex) {
-    DimensionColumnDataChunk dataChunk =
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex].convertToDimColDataChunk(page);
+  private int readSurrogatesFromColumnChunk(RawBlockletColumnChunks blockChunkHolder, int index,
+      int page, DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int chunkIndex) {
+    DimensionColumnPage dataChunk =
+        blockChunkHolder.getDimensionRawColumnChunks()[chunkIndex].decodeColumnPage(page);
     if (dimColumnEvaluatorInfo.getDimension().isColumnar()) {
       byte[] rawData = dataChunk.getChunkData(index);
       ByteBuffer byteBuffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
@@ -593,7 +595,7 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * @param dimColumnEvaluatorInfo
    * @return read surrogate of given row of given column group dimension
    */
-  private int readSurrogatesFromColumnGroupBlock(DimensionColumnDataChunk chunk, int index,
+  private int readSurrogatesFromColumnGroupBlock(DimensionColumnPage chunk, int index,
       DimColumnResolvedFilterInfo dimColumnEvaluatorInfo) {
     try {
       KeyStructureInfo keyStructureInfo =
@@ -601,9 +603,9 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
       byte[] colData = chunk.getChunkData(index);
       long[] result = keyStructureInfo.getKeyGenerator().getKeyArray(colData);
       int colGroupId =
-          QueryUtil.getColumnGroupId(segmentProperties, dimensionBlocksIndex[0]);
+          QueryUtil.getColumnGroupId(segmentProperties, dimensionChunkIndex[0]);
       return (int) result[segmentProperties
-          .getColumnGroupMdKeyOrdinal(colGroupId, dimensionBlocksIndex[0])];
+          .getColumnGroupMdKeyOrdinal(colGroupId, dimensionChunkIndex[0])];
     } catch (KeyGenException e) {
       LOGGER.error(e);
     }
@@ -611,33 +613,36 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
   }
 
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     bitSet.set(0);
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
       DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i);
       if (!dimColumnEvaluatorInfo.getDimension().getDataType().isComplexType()) {
-        if (null == blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]]) {
-          blockChunkHolder.getDimensionRawDataChunk()[dimensionBlocksIndex[i]] =
-              blockChunkHolder.getDataBlock()
-                  .getDimensionChunk(blockChunkHolder.getFileReader(), dimensionBlocksIndex[i]);
+        if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[i]])
+        {
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[dimensionChunkIndex[i]] =
+              rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                  rawBlockletColumnChunks.getFileReader(), dimensionChunkIndex[i]);
         }
       } else {
-        GenericQueryType complexType = complexDimensionInfoMap.get(dimensionBlocksIndex[i]);
-        complexType.fillRequiredBlockData(blockChunkHolder);
+        GenericQueryType complexType = complexDimensionInfoMap.get(dimensionChunkIndex[i]);
+        complexType.fillRequiredBlockData(rawBlockletColumnChunks);
       }
     }
 
     if (null != msrColEvalutorInfoList) {
       for (MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo : msrColEvalutorInfoList) {
-        if (null == blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]]) {
-          blockChunkHolder.getMeasureRawDataChunk()[measureBlocksIndex[0]] =
-              blockChunkHolder.getDataBlock()
-                  .getMeasureChunk(blockChunkHolder.getFileReader(), measureBlocksIndex[0]);
+        if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]]) {
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[measureChunkIndex[0]] =
+              rawBlockletColumnChunks.getDataBlock()
+                  .readMeasureChunk(rawBlockletColumnChunks.getFileReader(), measureChunkIndex[0]);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
index 3981211..1f63a81 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -32,12 +32,11 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -56,12 +55,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
    */
   private boolean isDefaultValuePresentInFilter;
   private int lastDimensionColOrdinal = 0;
-  public RowLevelRangeGrtThanFiterExecuterImpl(
-      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
+
+  RowLevelRangeGrtThanFiterExecuterImpl(List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
-      Object[] msrFilterRangeValues,
-      SegmentProperties segmentProperties) {
+      Object[] msrFilterRangeValues, SegmentProperties segmentProperties) {
     super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties,
         null);
     this.filterRangeValues = filterRangeValues;
@@ -111,17 +109,18 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     }
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     BitSet bitSet = new BitSet(1);
     boolean isScanRequired = false;
     byte[] maxValue = null;
     if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        maxValue = blockMaxValue[measureBlocksIndex[0] + lastDimensionColOrdinal];
+        maxValue = blockMaxValue[measureChunkIndex[0] + lastDimensionColOrdinal];
         isScanRequired =
             isScanRequired(maxValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType());
       } else {
-        maxValue = blockMaxValue[dimensionBlocksIndex[0]];
+        maxValue = blockMaxValue[dimensionChunkIndex[0]];
         isScanRequired = isScanRequired(maxValue, filterRangeValues);
       }
     } else {
@@ -168,24 +167,25 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+      int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
       return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+          .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
               numberOfRows, true);
     }
     if (isDimensionPresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk rawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMaxValues() != null) {
@@ -197,26 +197,28 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
               bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             } else {
-              BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+              BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             }
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
               rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]);
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-      MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+      MeasureRawColumnChunk rawColumnChunk =
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMaxValues() != null) {
@@ -226,7 +228,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
                 .getMeasureObjectFromDataType(rawColumnChunk.getMinValues()[i],
                     msrColEvalutorInfoList.get(0).getType()));
             ColumnPage columnPage =
-                rawColumnChunk.convertToColumnPage(i);
+                rawColumnChunk.decodeColumnPage(i);
             if (compare < 0 && columnPage.getNullBits().isEmpty()) {
               BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]);
               bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
@@ -239,7 +241,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
@@ -296,16 +298,16 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     return bitSet;
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = null;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows);
     }
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet,
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
           CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
     }
     return bitSet;
@@ -317,12 +319,12 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
     int last = 0;
@@ -330,11 +332,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     byte[][] filterValues = this.filterRangeValues;
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i], true);
       if (start >= 0) {
         start = CarbonUtil
-            .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[i],
+            .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[i],
                 numerOfRows);
       }
       // Logic will handle the case where the range filter member is not present in block
@@ -350,7 +352,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
         // 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(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             > 0) {
           start = start + 1;
         }
@@ -358,7 +360,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
 
       last = start;
       for (int j = start; j < numerOfRows; j++) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
         last++;
       }
       startIndex = last;
@@ -376,11 +378,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
    * 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 dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = this.filterRangeValues;
@@ -391,11 +393,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
       int startIndex = 0;
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, filterValues[k], true);
         if (start >= 0) {
           start = CarbonUtil
-              .nextGreaterValueToTarget(start, dimensionColumnDataChunk, filterValues[k],
+              .nextGreaterValueToTarget(start, dimensionColumnPage, filterValues[k],
                   numerOfRows);
         }
         if (start < 0) {
@@ -406,7 +408,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
           // 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) {
+          if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) > 0) {
             start = start + 1;
           }
         }
@@ -423,7 +425,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     } else {
       for (int k = 0; k < filterValues.length; k++) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) > 0) {
+          if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) > 0) {
             bitSet.set(i);
           }
         }
@@ -432,21 +434,24 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     if (isDimensionPresentInCurrentBlock[0]) {
       if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        super.readBlocks(blockChunkHolder);
+        super.readColumnChunks(rawBlockletColumnChunks);
       }
-      int blockIndex = dimensionBlocksIndex[0];
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = dimensionChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex = measureBlocksIndex[0];
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = measureChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
index f2ddcb6..9140a11 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -32,12 +32,11 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -47,7 +46,7 @@ import org.apache.carbondata.core.util.comparator.SerializableComparator;
 
 public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilterExecuterImpl {
 
-  protected byte[][] filterRangeValues;
+  private byte[][] filterRangeValues;
   private Object[] msrFilterRangeValues;
   private SerializableComparator comparator;
   /**
@@ -56,7 +55,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
   private boolean isDefaultValuePresentInFilter;
   private int lastDimensionColOrdinal = 0;
 
-  public RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
+  RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
@@ -116,11 +115,11 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     byte[] maxValue = null;
     if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        maxValue = blockMaxValue[measureBlocksIndex[0] + lastDimensionColOrdinal];
+        maxValue = blockMaxValue[measureChunkIndex[0] + lastDimensionColOrdinal];
         isScanRequired =
             isScanRequired(maxValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType());
       } else {
-        maxValue = blockMaxValue[dimensionBlocksIndex[0]];
+        maxValue = blockMaxValue[dimensionChunkIndex[0]];
         isScanRequired = isScanRequired(maxValue, filterRangeValues);
       }
     } else {
@@ -166,25 +165,26 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+      int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
       return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+          .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
               numberOfRows, true);
     }
 
     if (isDimensionPresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk rawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMaxValues() != null) {
@@ -196,26 +196,28 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
               bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             } else {
-              BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+              BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             }
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
               rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
-    } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]);
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+    } else {
+      int chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-      MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+      MeasureRawColumnChunk rawColumnChunk =
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMaxValues() != null) {
@@ -225,28 +227,27 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
                 .getMeasureObjectFromDataType(rawColumnChunk.getMinValues()[i],
                     msrColEvalutorInfoList.get(0).getType()));
             ColumnPage columnPage =
-                rawColumnChunk.convertToColumnPage(i);
+                rawColumnChunk.decodeColumnPage(i);
             if (compare <= 0 && columnPage.getNullBits().isEmpty()) {
               BitSet bitSet = new BitSet(rawColumnChunk.getRowCount()[i]);
               bitSet.flip(0, rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             } else {
               BitSet bitSet =
-                  getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+                  getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                       rawColumnChunk.getRowCount()[i]);
               bitSetGroup.setBitSet(bitSet, i);
             }
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     }
-    return null;
   }
 
   @Override
@@ -297,16 +298,16 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
   }
 
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = null;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows);
+    if (dimensionColumnPage.isExplicitSorted()) {
+      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows);
     }
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet,
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
           CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
     }
     return bitSet;
@@ -318,12 +319,12 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      DimensionColumnPage dimensionColumnPage, int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
     int last = 0;
@@ -331,7 +332,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     byte[][] filterValues = this.filterRangeValues;
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i], false);
       if (start < 0) {
         start = -(start + 1);
@@ -342,14 +343,14 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
         // 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(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             > 0) {
           start = start + 1;
         }
       }
       last = start;
       for (int j = start; j < numerOfRows; j++) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
         last++;
       }
       startIndex = last;
@@ -366,11 +367,11 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
    * 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 dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = this.filterRangeValues;
@@ -381,7 +382,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
       int startIndex = 0;
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, filterValues[k], false);
         if (start < 0) {
           start = -(start + 1);
@@ -391,7 +392,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)) > 0) {
+          if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) > 0) {
             start = start + 1;
           }
         }
@@ -409,7 +410,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     } else {
       for (int k = 0; k < filterValues.length; k++) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) >= 0) {
+          if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) >= 0) {
             bitSet.set(i);
           }
         }
@@ -418,21 +419,24 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     if (isDimensionPresentInCurrentBlock[0]) {
       if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        super.readBlocks(blockChunkHolder);
+        super.readColumnChunks(rawBlockletColumnChunks);
       }
-      int blockIndex = dimensionBlocksIndex[0];
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = dimensionChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex = measureBlocksIndex[0];
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = measureChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
index a44bc1a..120671f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -35,12 +35,11 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -118,11 +117,11 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     boolean isScanRequired = false;
     if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        minValue = blockMinValue[measureBlocksIndex[0] + lastDimensionColOrdinal];
+        minValue = blockMinValue[measureChunkIndex[0] + lastDimensionColOrdinal];
         isScanRequired =
             isScanRequired(minValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType());
       } else {
-        minValue = blockMinValue[dimensionBlocksIndex[0]];
+        minValue = blockMinValue[dimensionChunkIndex[0]];
         isScanRequired = isScanRequired(minValue, filterRangeValues);
       }
     } else {
@@ -168,60 +167,63 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+      int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
       return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+          .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
               numberOfRows, true);
     }
     if (isDimensionPresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk rawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMinValues() != null) {
           if (isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues)) {
-            BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+            BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                 rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
               rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]);
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-      MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+      MeasureRawColumnChunk rawColumnChunk =
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMinValues() != null) {
           if (isScanRequired(rawColumnChunk.getMinValues()[i], this.msrFilterRangeValues,
               msrColEvalutorInfoList.get(0).getType())) {
             BitSet bitSet =
-                getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+                getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                     rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
@@ -278,7 +280,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     return bitSet;
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     byte[] defaultValue = null;
     if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
@@ -287,7 +289,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
               dimColEvaluatorInfoList.get(0).getDimension().getDataType());
       int key = directDictionaryGenerator.generateDirectSurrogateKey(null);
       CarbonDimension currentBlockDimension =
-          segmentProperties.getDimensions().get(dimensionBlocksIndex[0]);
+          segmentProperties.getDimensions().get(dimensionChunkIndex[0]);
       if (currentBlockDimension.isSortColumn()) {
         defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
             this.segmentProperties.getSortColumnsGenerator());
@@ -298,14 +300,14 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
     BitSet bitSet = null;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows,
+    if (dimensionColumnPage.isExplicitSorted()) {
+      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows,
           defaultValue);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows, defaultValue);
     }
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet,
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
           CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
     }
     return bitSet;
@@ -317,12 +319,12 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows,
+      DimensionColumnPage dimensionColumnPage, int numerOfRows,
       byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
@@ -333,7 +335,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     //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,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               defaultValue, true);
       if (start < 0) {
         skip = -(start + 1);
@@ -350,7 +352,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     }
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i], true);
       if (start < 0) {
         start = -(start + 1);
@@ -361,14 +363,14 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
         // will be pointing to the next consecutive position. So compare it again and point to the
         // previous value returned from getFirstIndexUsingBinarySearch.
         if (ByteUtil.compare(filterValues[i],
-            dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             < 0) {
           start = start - 1;
         }
       }
       last = start;
       for (int j = start; j >= skip; j--) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
         last--;
       }
       startIndex = last;
@@ -385,12 +387,12 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
    * 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 dimensionColumnPage
    * @param numerOfRows
    * @param defaultValue
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows, byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = this.filterRangeValues;
@@ -403,7 +405,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
       //find the number of default values to skip the null value in case of direct dictionary
       if (null != defaultValue) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, defaultValue, true);
         if (start < 0) {
           skip = -(start + 1);
@@ -420,7 +422,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
       }
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, filterValues[k], true);
         if (start < 0) {
           start = -(start + 1);
@@ -430,7 +432,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
           // When negative value of start is returned from getFirstIndexUsingBinarySearch the Start
           // will be pointing to the next consecutive position. So compare it again and point to the
           // previous value returned from getFirstIndexUsingBinarySearch.
-          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) < 0) {
+          if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) < 0) {
             start = start - 1;
           }
         }
@@ -447,7 +449,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     } else {
       for (int k = 0; k < filterValues.length; k++) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) <= 0) {
+          if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) <= 0) {
             bitSet.set(i);
           }
         }
@@ -456,21 +458,24 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks)
+      throws IOException {
     if (isDimensionPresentInCurrentBlock[0]) {
       if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        super.readBlocks(blockChunkHolder);
+        super.readColumnChunks(rawBlockletColumnChunks);
       }
-      int blockIndex = dimensionBlocksIndex[0];
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = dimensionChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex = measureBlocksIndex[0];
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = measureChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }


[39/49] carbondata git commit: [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
index 8d394db..e69de29 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -1,610 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.util
-
-import java.text.SimpleDateFormat
-import java.util
-import java.util.{Date, List, Locale}
-
-import scala.collection.{immutable, mutable}
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.commons.lang3.StringUtils
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
-import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
-import org.apache.hadoop.mapreduce.task.{JobContextImpl, TaskAttemptContextImpl}
-import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.GenericInternalRow
-import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
-import org.apache.spark.sql.util.CarbonException
-import org.apache.spark.sql.util.SparkSQLUtil.sessionState
-
-import org.apache.carbondata.common.constants.LoggerAction
-import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
-import org.apache.carbondata.core.indexstore.PartitionSpec
-import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.processing.util.{CarbonLoaderUtil, DeleteLoadFolders, TableOptionConstant}
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.load.DataLoadProcessBuilderOnSpark.LOGGER
-import org.apache.carbondata.spark.load.ValidateUtil
-import org.apache.carbondata.spark.rdd.SerializableConfiguration
-
-/**
- * the util object of data loading
- */
-object DataLoadingUtil {
-
-  val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  /**
-   * get data loading options and initialise default value
-   */
-  def getDataLoadingOptions(
-      carbonProperty: CarbonProperties,
-      options: immutable.Map[String, String]): mutable.Map[String, String] = {
-    val optionsFinal = scala.collection.mutable.Map[String, String]()
-    optionsFinal.put("delimiter", options.getOrElse("delimiter", ","))
-    optionsFinal.put("quotechar", options.getOrElse("quotechar", "\""))
-    optionsFinal.put("fileheader", options.getOrElse("fileheader", ""))
-    optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
-    optionsFinal.put("columndict", options.getOrElse("columndict", null))
-
-    optionsFinal.put("escapechar",
-      CarbonLoaderUtil.getEscapeChar(options.getOrElse("escapechar", "\\")))
-
-    optionsFinal.put(
-      "serialization_null_format",
-      options.getOrElse("serialization_null_format", "\\N"))
-
-    optionsFinal.put(
-      "bad_records_logger_enable",
-      options.getOrElse(
-        "bad_records_logger_enable",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE,
-          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT)))
-
-    val badRecordActionValue = carbonProperty.getProperty(
-      CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
-      CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT)
-
-    optionsFinal.put(
-      "bad_records_action",
-      options.getOrElse(
-        "bad_records_action",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION,
-          badRecordActionValue)))
-
-    optionsFinal.put(
-      "is_empty_data_bad_record",
-      options.getOrElse(
-        "is_empty_data_bad_record",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD,
-          CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT)))
-
-    optionsFinal.put(
-      "skip_empty_line",
-      options.getOrElse(
-        "skip_empty_line",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_SKIP_EMPTY_LINE)))
-
-    optionsFinal.put("all_dictionary_path", options.getOrElse("all_dictionary_path", ""))
-
-    optionsFinal.put(
-      "complex_delimiter_level_1",
-      options.getOrElse("complex_delimiter_level_1", "$"))
-
-    optionsFinal.put(
-      "complex_delimiter_level_2",
-      options.getOrElse("complex_delimiter_level_2", ":"))
-
-    optionsFinal.put(
-      "dateformat",
-      options.getOrElse(
-        "dateformat",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT,
-          CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)))
-
-    optionsFinal.put(
-      "timestampformat",
-      options.getOrElse(
-        "timestampformat",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_TIMESTAMPFORMAT,
-          CarbonLoadOptionConstants.CARBON_OPTIONS_TIMESTAMPFORMAT_DEFAULT)))
-
-    optionsFinal.put(
-      "global_sort_partitions",
-      options.getOrElse(
-        "global_sort_partitions",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS,
-          null)))
-
-    optionsFinal.put("maxcolumns", options.getOrElse("maxcolumns", null))
-
-    optionsFinal.put(
-      "batch_sort_size_inmb",
-      options.getOrElse(
-        "batch_sort_size_inmb",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB,
-          carbonProperty.getProperty(
-            CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
-            CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))))
-
-    optionsFinal.put(
-      "bad_record_path",
-      options.getOrElse(
-        "bad_record_path",
-        carbonProperty.getProperty(
-          CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
-          carbonProperty.getProperty(
-            CarbonCommonConstants.CARBON_BADRECORDS_LOC,
-            CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))))
-
-    val useOnePass = options.getOrElse(
-      "single_pass",
-      carbonProperty.getProperty(
-        CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
-        CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT)).trim.toLowerCase match {
-      case "true" =>
-        true
-      case "false" =>
-        // when single_pass = false  and if either alldictionarypath
-        // or columnDict is configured the do not allow load
-        if (StringUtils.isNotEmpty(optionsFinal("all_dictionary_path")) ||
-            StringUtils.isNotEmpty(optionsFinal("columndict"))) {
-          throw new MalformedCarbonCommandException(
-            "Can not use all_dictionary_path or columndict without single_pass.")
-        } else {
-          false
-        }
-      case illegal =>
-        val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-        LOGGER.error(s"Can't use single_pass, because illegal syntax found: [$illegal] " +
-                     "Please set it as 'true' or 'false'")
-        false
-    }
-    optionsFinal.put("single_pass", useOnePass.toString)
-    optionsFinal
-  }
-
-  /**
-   * check whether using default value or not
-   */
-  private def checkDefaultValue(value: String, default: String) = {
-    if (StringUtils.isEmpty(value)) {
-      default
-    } else {
-      value
-    }
-  }
-
-  /**
-   * build CarbonLoadModel for data loading
-   * @param table CarbonTable object containing all metadata information for the table
-   *              like table name, table path, schema, etc
-   * @param options Load options from user input
-   * @return a new CarbonLoadModel instance
-   */
-  def buildCarbonLoadModelJava(
-      table: CarbonTable,
-      options: java.util.Map[String, String]
-  ): CarbonLoadModel = {
-    val carbonProperty: CarbonProperties = CarbonProperties.getInstance
-    val optionsFinal = getDataLoadingOptions(carbonProperty, options.asScala.toMap)
-    optionsFinal.put("sort_scope", "no_sort")
-    if (!options.containsKey("fileheader")) {
-      val csvHeader = table.getCreateOrderColumn(table.getTableName)
-        .asScala.map(_.getColName).mkString(",")
-      optionsFinal.put("fileheader", csvHeader)
-    }
-    val model = new CarbonLoadModel()
-    buildCarbonLoadModel(
-      table = table,
-      carbonProperty = carbonProperty,
-      options = options.asScala.toMap,
-      optionsFinal = optionsFinal,
-      carbonLoadModel = model,
-      hadoopConf = null)  // we have provided 'fileheader', so it can be null
-
-    // set default values
-    model.setTimestampformat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-    model.setDateFormat(CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-    model.setUseOnePass(options.asScala.getOrElse("onepass", "false").toBoolean)
-    model.setDictionaryServerHost(options.asScala.getOrElse("dicthost", null))
-    model.setDictionaryServerPort(options.asScala.getOrElse("dictport", "-1").toInt)
-    model
-  }
-
-  /**
-   * build CarbonLoadModel for data loading
-   * @param table CarbonTable object containing all metadata information for the table
-   *              like table name, table path, schema, etc
-   * @param carbonProperty Carbon property instance
-   * @param options Load options from user input
-   * @param optionsFinal Load options that populated with default values for optional options
-   * @param carbonLoadModel The output load model
-   * @param hadoopConf hadoopConf is needed to read CSV header if there 'fileheader' is not set in
-   *                   user provided load options
-   */
-  def buildCarbonLoadModel(
-      table: CarbonTable,
-      carbonProperty: CarbonProperties,
-      options: immutable.Map[String, String],
-      optionsFinal: mutable.Map[String, String],
-      carbonLoadModel: CarbonLoadModel,
-      hadoopConf: Configuration,
-      partition: Map[String, Option[String]] = Map.empty,
-      isDataFrame: Boolean = false): Unit = {
-    carbonLoadModel.setTableName(table.getTableName)
-    carbonLoadModel.setDatabaseName(table.getDatabaseName)
-    carbonLoadModel.setTablePath(table.getTablePath)
-    carbonLoadModel.setTableName(table.getTableName)
-    val dataLoadSchema = new CarbonDataLoadSchema(table)
-    // Need to fill dimension relation
-    carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-    val sort_scope = optionsFinal("sort_scope")
-    val single_pass = optionsFinal("single_pass")
-    val bad_records_logger_enable = optionsFinal("bad_records_logger_enable")
-    val bad_records_action = optionsFinal("bad_records_action")
-    var bad_record_path = optionsFinal("bad_record_path")
-    val global_sort_partitions = optionsFinal("global_sort_partitions")
-    val timestampformat = optionsFinal("timestampformat")
-    val dateFormat = optionsFinal("dateformat")
-    val delimeter = optionsFinal("delimiter")
-    val complex_delimeter_level1 = optionsFinal("complex_delimiter_level_1")
-    val complex_delimeter_level2 = optionsFinal("complex_delimiter_level_2")
-    val all_dictionary_path = optionsFinal("all_dictionary_path")
-    val column_dict = optionsFinal("columndict")
-    ValidateUtil.validateDateTimeFormat(timestampformat, "TimestampFormat")
-    ValidateUtil.validateDateTimeFormat(dateFormat, "DateFormat")
-    ValidateUtil.validateSortScope(table, sort_scope)
-
-    if (bad_records_logger_enable.toBoolean ||
-        LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
-      if (!CarbonUtil.isValidBadStorePath(bad_record_path)) {
-        CarbonException.analysisException("Invalid bad records location.")
-      }
-      bad_record_path = CarbonUtil.checkAndAppendHDFSUrl(bad_record_path)
-    }
-    carbonLoadModel.setBadRecordsLocation(bad_record_path)
-
-    ValidateUtil.validateGlobalSortPartitions(global_sort_partitions)
-    carbonLoadModel.setEscapeChar(checkDefaultValue(optionsFinal("escapechar"), "\\"))
-    carbonLoadModel.setQuoteChar(checkDefaultValue(optionsFinal("quotechar"), "\""))
-    carbonLoadModel.setCommentChar(checkDefaultValue(optionsFinal("commentchar"), "#"))
-
-    // if there isn't file header in csv file and load sql doesn't provide FILEHEADER option,
-    // we should use table schema to generate file header.
-    var fileHeader = optionsFinal("fileheader")
-    val headerOption = options.get("header")
-    if (headerOption.isDefined) {
-      // whether the csv file has file header
-      // the default value is true
-      val header = try {
-        headerOption.get.toBoolean
-      } catch {
-        case ex: IllegalArgumentException =>
-          throw new MalformedCarbonCommandException(
-            "'header' option should be either 'true' or 'false'. " + ex.getMessage)
-      }
-      if (header) {
-        if (fileHeader.nonEmpty) {
-          throw new MalformedCarbonCommandException(
-            "When 'header' option is true, 'fileheader' option is not required.")
-        }
-      } else {
-        if (fileHeader.isEmpty) {
-          fileHeader = table.getCreateOrderColumn(table.getTableName)
-            .asScala.map(_.getColName).mkString(",")
-        }
-      }
-    }
-
-    carbonLoadModel.setTimestampformat(timestampformat)
-    carbonLoadModel.setDateFormat(dateFormat)
-    carbonLoadModel.setDefaultTimestampFormat(carbonProperty.getProperty(
-      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-
-    carbonLoadModel.setDefaultDateFormat(carbonProperty.getProperty(
-      CarbonCommonConstants.CARBON_DATE_FORMAT,
-      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-
-    carbonLoadModel.setSerializationNullFormat(
-        TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName + "," +
-        optionsFinal("serialization_null_format"))
-
-    carbonLoadModel.setBadRecordsLoggerEnable(
-        TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName + "," + bad_records_logger_enable)
-
-    carbonLoadModel.setBadRecordsAction(
-        TableOptionConstant.BAD_RECORDS_ACTION.getName + "," + bad_records_action.toUpperCase)
-
-    carbonLoadModel.setIsEmptyDataBadRecord(
-        DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," +
-        optionsFinal("is_empty_data_bad_record"))
-
-    carbonLoadModel.setSkipEmptyLine(optionsFinal("skip_empty_line"))
-
-    carbonLoadModel.setSortScope(sort_scope)
-    carbonLoadModel.setBatchSortSizeInMb(optionsFinal("batch_sort_size_inmb"))
-    carbonLoadModel.setGlobalSortPartitions(global_sort_partitions)
-    carbonLoadModel.setUseOnePass(single_pass.toBoolean)
-
-    if (delimeter.equalsIgnoreCase(complex_delimeter_level1) ||
-        complex_delimeter_level1.equalsIgnoreCase(complex_delimeter_level2) ||
-        delimeter.equalsIgnoreCase(complex_delimeter_level2)) {
-      CarbonException.analysisException(s"Field Delimiter and Complex types delimiter are same")
-    } else {
-      carbonLoadModel.setComplexDelimiterLevel1(complex_delimeter_level1)
-      carbonLoadModel.setComplexDelimiterLevel2(complex_delimeter_level2)
-    }
-    // set local dictionary path, and dictionary file extension
-    carbonLoadModel.setAllDictPath(all_dictionary_path)
-    carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimeter))
-    carbonLoadModel.setCsvHeader(fileHeader)
-    carbonLoadModel.setColDictFilePath(column_dict)
-
-    val ignoreColumns = new util.ArrayList[String]()
-    if (!isDataFrame) {
-      ignoreColumns.addAll(partition.filter(_._2.isDefined).keys.toList.asJava)
-    }
-    carbonLoadModel.setCsvHeaderColumns(
-      CommonUtil.getCsvHeaderColumns(carbonLoadModel, hadoopConf, ignoreColumns))
-
-    val validatedMaxColumns = CommonUtil.validateMaxColumns(
-      carbonLoadModel.getCsvHeaderColumns,
-      optionsFinal("maxcolumns"))
-
-    carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
-    if (null == carbonLoadModel.getLoadMetadataDetails) {
-      CommonUtil.readLoadMetadataDetails(carbonLoadModel)
-    }
-  }
-
-  private def isLoadDeletionRequired(metaDataLocation: String): Boolean = {
-    val details = SegmentStatusManager.readLoadMetadata(metaDataLocation)
-    if (details != null && details.nonEmpty) for (oneRow <- details) {
-      if ((SegmentStatus.MARKED_FOR_DELETE == oneRow.getSegmentStatus ||
-           SegmentStatus.COMPACTED == oneRow.getSegmentStatus ||
-           SegmentStatus.INSERT_IN_PROGRESS == oneRow.getSegmentStatus ||
-           SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneRow.getSegmentStatus) &&
-          oneRow.getVisibility.equalsIgnoreCase("true")) {
-        return true
-      }
-    }
-    false
-  }
-
-  def deleteLoadsAndUpdateMetadata(
-      isForceDeletion: Boolean,
-      carbonTable: CarbonTable,
-      specs: util.List[PartitionSpec]): Unit = {
-    if (isLoadDeletionRequired(carbonTable.getMetadataPath)) {
-      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-
-      val (details, updationRequired) =
-        isUpdationRequired(
-          isForceDeletion,
-          carbonTable,
-          absoluteTableIdentifier)
-
-
-      if (updationRequired) {
-        val carbonTableStatusLock =
-          CarbonLockFactory.getCarbonLockObj(
-            absoluteTableIdentifier,
-            LockUsage.TABLE_STATUS_LOCK
-          )
-        var locked = false
-        var updationCompletionStaus = false
-        try {
-          // Update load metadate file after cleaning deleted nodes
-          locked = carbonTableStatusLock.lockWithRetries()
-          if (locked) {
-            LOGGER.info("Table status lock has been successfully acquired.")
-            // Again read status and check to verify updation required or not.
-            val (details, updationRequired) =
-              isUpdationRequired(
-                isForceDeletion,
-                carbonTable,
-                absoluteTableIdentifier)
-            if (!updationRequired) {
-              return
-            }
-            // read latest table status again.
-            val latestMetadata = SegmentStatusManager
-              .readLoadMetadata(carbonTable.getMetadataPath)
-
-            // update the metadata details from old to new status.
-            val latestStatus = CarbonLoaderUtil
-              .updateLoadMetadataFromOldToNew(details, latestMetadata)
-
-            CarbonLoaderUtil.writeLoadMetadata(absoluteTableIdentifier, latestStatus)
-          } else {
-            val dbName = absoluteTableIdentifier.getCarbonTableIdentifier.getDatabaseName
-            val tableName = absoluteTableIdentifier.getCarbonTableIdentifier.getTableName
-            val errorMsg = "Clean files request is failed for " +
-                           s"$dbName.$tableName" +
-                           ". Not able to acquire the table status lock due to other operation " +
-                           "running in the background."
-            LOGGER.audit(errorMsg)
-            LOGGER.error(errorMsg)
-            throw new Exception(errorMsg + " Please try after some time.")
-          }
-          updationCompletionStaus = true
-        } finally {
-          if (locked) {
-            CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.TABLE_STATUS_LOCK)
-          }
-        }
-        if (updationCompletionStaus) {
-          DeleteLoadFolders
-            .physicalFactAndMeasureMetadataDeletion(absoluteTableIdentifier,
-              carbonTable.getMetadataPath, isForceDeletion, specs)
-        }
-      }
-    }
-  }
-
-  private def isUpdationRequired(isForceDeletion: Boolean,
-      carbonTable: CarbonTable,
-      absoluteTableIdentifier: AbsoluteTableIdentifier): (Array[LoadMetadataDetails], Boolean) = {
-    val details = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
-    // Delete marked loads
-    val isUpdationRequired =
-      DeleteLoadFolders.deleteLoadFoldersFromFileSystem(
-        absoluteTableIdentifier,
-        isForceDeletion,
-        details,
-        carbonTable.getMetadataPath
-      )
-    (details, isUpdationRequired)
-  }
-
-  /**
-   * creates a RDD that does reading of multiple CSV files
-   */
-  def csvFileScanRDD(
-      spark: SparkSession,
-      model: CarbonLoadModel,
-      hadoopConf: Configuration
-  ): RDD[InternalRow] = {
-    // 1. partition
-    val defaultMaxSplitBytes = sessionState(spark).conf.filesMaxPartitionBytes
-    val openCostInBytes = sessionState(spark).conf.filesOpenCostInBytes
-    val defaultParallelism = spark.sparkContext.defaultParallelism
-    CommonUtil.configureCSVInputFormat(hadoopConf, model)
-    hadoopConf.set(FileInputFormat.INPUT_DIR, model.getFactFilePath)
-    val jobConf = new JobConf(hadoopConf)
-    SparkHadoopUtil.get.addCredentials(jobConf)
-    val jobContext = new JobContextImpl(jobConf, null)
-    val inputFormat = new CSVInputFormat()
-    val rawSplits = inputFormat.getSplits(jobContext).toArray
-    val splitFiles = rawSplits.map { split =>
-      val fileSplit = split.asInstanceOf[FileSplit]
-      PartitionedFile(
-        InternalRow.empty,
-        fileSplit.getPath.toString,
-        fileSplit.getStart,
-        fileSplit.getLength,
-        fileSplit.getLocations)
-    }.sortBy(_.length)(implicitly[Ordering[Long]].reverse)
-    val totalBytes = splitFiles.map(_.length + openCostInBytes).sum
-    val bytesPerCore = totalBytes / defaultParallelism
-
-    val maxSplitBytes = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore))
-    LOGGER.info(s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " +
-                s"open cost is considered as scanning $openCostInBytes bytes.")
-
-    val partitions = new ArrayBuffer[FilePartition]
-    val currentFiles = new ArrayBuffer[PartitionedFile]
-    var currentSize = 0L
-
-    def closePartition(): Unit = {
-      if (currentFiles.nonEmpty) {
-        val newPartition =
-          FilePartition(
-            partitions.size,
-            currentFiles.toArray.toSeq)
-        partitions += newPartition
-      }
-      currentFiles.clear()
-      currentSize = 0
-    }
-
-    splitFiles.foreach { file =>
-      if (currentSize + file.length > maxSplitBytes) {
-        closePartition()
-      }
-      // Add the given file to the current partition.
-      currentSize += file.length + openCostInBytes
-      currentFiles += file
-    }
-    closePartition()
-
-    // 2. read function
-    val serializableConfiguration = new SerializableConfiguration(jobConf)
-    val readFunction = new (PartitionedFile => Iterator[InternalRow]) with Serializable {
-      override def apply(file: PartitionedFile): Iterator[InternalRow] = {
-        new Iterator[InternalRow] {
-          val hadoopConf = serializableConfiguration.value
-          val jobTrackerId: String = {
-            val formatter = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US)
-            formatter.format(new Date())
-          }
-          val attemptId = new TaskAttemptID(jobTrackerId, 0, TaskType.MAP, 0, 0)
-          val hadoopAttemptContext = new TaskAttemptContextImpl(hadoopConf, attemptId)
-          val inputSplit =
-            new FileSplit(new Path(file.filePath), file.start, file.length, file.locations)
-          var finished = false
-          val inputFormat = new CSVInputFormat()
-          val reader = inputFormat.createRecordReader(inputSplit, hadoopAttemptContext)
-          reader.initialize(inputSplit, hadoopAttemptContext)
-
-          override def hasNext: Boolean = {
-            if (!finished) {
-              if (reader != null) {
-                if (reader.nextKeyValue()) {
-                  true
-                } else {
-                  finished = true
-                  reader.close()
-                  false
-                }
-              } else {
-                finished = true
-                false
-              }
-            } else {
-              false
-            }
-          }
-
-          override def next(): InternalRow = {
-            new GenericInternalRow(reader.getCurrentValue.get().asInstanceOf[Array[Any]])
-          }
-        }
-      }
-    }
-    new FileScanRDD(spark, readFunction, partitions)
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index 2bd4f45..fb9ecac 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -321,7 +321,7 @@ object GlobalDictionaryUtil {
       carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
     // get load count
     if (null == carbonLoadModel.getLoadMetadataDetails) {
-      CommonUtil.readLoadMetadataDetails(carbonLoadModel)
+      carbonLoadModel.readAndSetLoadMetadataDetails()
     }
     val absoluteTableIdentifier = AbsoluteTableIdentifier.from(carbonLoadModel.getTablePath, table)
     DictionaryLoadModel(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index 84215fd..b1e4083 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -34,6 +34,7 @@ import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.PartitionUtils
 
 import org.apache.carbondata.common.constants.LoggerAction
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.datatype.DataTypes
@@ -42,7 +43,6 @@ import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.processing.util.CarbonLoaderUtil
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.{CommonUtil, DataTypeConverterUtil}
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 8d3110a..6f8155c 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -153,7 +153,7 @@ object CarbonDataRDDFactory {
     if (compactionModel.compactionType != CompactionType.IUD_UPDDEL_DELTA) {
       // update the updated table status. For the case of Update Delta Compaction the Metadata
       // is filled in LoadModel, no need to refresh.
-      CommonUtil.readLoadMetadataDetails(carbonLoadModel)
+      carbonLoadModel.readAndSetLoadMetadataDetails()
     }
 
     val compactionThread = new Thread {
@@ -274,7 +274,7 @@ object CarbonDataRDDFactory {
     loadModel.setTableName(table.getCarbonTableIdentifier.getTableName)
     loadModel.setDatabaseName(table.getCarbonTableIdentifier.getDatabaseName)
     loadModel.setTablePath(table.getTablePath)
-    CommonUtil.readLoadMetadataDetails(loadModel)
+    loadModel.readAndSetLoadMetadataDetails()
     val loadStartTime = CarbonUpdateUtil.readCurrentTime()
     loadModel.setFactTimeStamp(loadStartTime)
     loadModel

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
index e1bef9c..47d918e 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
@@ -76,7 +76,7 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
       }
 
       // scan again and determine if anything is there to merge again.
-      CommonUtil.readLoadMetadataDetails(carbonLoadModel)
+      carbonLoadModel.readAndSetLoadMetadataDetails()
       segList = carbonLoadModel.getLoadMetadataDetails
       // in case of major compaction we will scan only once and come out as it will keep
       // on doing major for the new loads also.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
index 7d70534..adf5e04 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSource.scala
@@ -35,13 +35,13 @@ import org.apache.spark.sql.streaming.OutputMode
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.util.CarbonException
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
 import org.apache.carbondata.core.metadata.schema.table.TableInfo
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.spark.CarbonOption
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 import org.apache.carbondata.streaming.{CarbonStreamException, StreamSinkFactory}
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
index 0fd5437..4e1ae56 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonCreateDataMapCommand.scala
@@ -23,10 +23,10 @@ import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.command.preaaggregate.CreatePreAggregateTableCommand
 import org.apache.spark.sql.execution.command.timeseries.TimeSeriesUtil
 
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider._
-import org.apache.carbondata.spark.exception.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 
 /**
  * Below command class will be used to create datamap on table

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
index 2675036..dcc71a2 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/datamap/CarbonDropDataMapCommand.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.execution.command.AtomicRunnableCommand
 import org.apache.spark.sql.execution.command.preaaggregate.PreAggregateUtil
 import org.apache.spark.sql.execution.command.table.CarbonDropTableCommand
 
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, NoSuchDataMapException}
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
@@ -34,7 +35,6 @@ import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.events._
-import org.apache.carbondata.spark.exception.{MalformedCarbonCommandException, NoSuchDataMapException}
 
 /**
  * Drops the datamap and any related tables associated with the datamap

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
index 4645f98..75de1fe 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonAlterTableCompactionCommand.scala
@@ -21,15 +21,17 @@ import java.io.{File, IOException}
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession, SQLContext}
+import org.apache.spark.sql.{CarbonEnv, Row, SQLContext, SparkSession}
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.execution.command.{AlterTableModel, AtomicRunnableCommand, CarbonMergerMapping, CompactionModel, DataCommand}
+import org.apache.spark.sql.execution.command.{AlterTableModel, AtomicRunnableCommand, CarbonMergerMapping, CompactionModel}
 import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionCatalog}
 import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.AlterTableUtil
 
+import org.apache.carbondata.common.exceptions.ConcurrentOperationException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -44,7 +46,6 @@ import org.apache.carbondata.events._
 import org.apache.carbondata.processing.loading.events.LoadEvents.LoadMetadataEvent
 import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
-import org.apache.carbondata.spark.exception.{ConcurrentOperationException, MalformedCarbonCommandException}
 import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
 import org.apache.carbondata.spark.util.CommonUtil
 import org.apache.carbondata.streaming.StreamHandoffRDD
@@ -90,7 +91,16 @@ case class CarbonAlterTableCompactionCommand(
   }
 
   override def processData(sparkSession: SparkSession): Seq[Row] = {
-    operationContext.setProperty("compactionException", "true")
+    val LOGGER: LogService =
+      LogServiceFactory.getLogService(this.getClass.getName)
+    val isLoadInProgress = SegmentStatusManager.checkIfAnyLoadInProgressForTable(table)
+    if (isLoadInProgress) {
+      val message = "Cannot run data loading and compaction on same table concurrently. " +
+                    "Please wait for load to finish"
+      LOGGER.error(message)
+      throw new ConcurrentOperationException(message)
+    }
+
     var compactionType: CompactionType = null
     var compactionException = "true"
     try {
@@ -103,7 +113,6 @@ case class CarbonAlterTableCompactionCommand(
           .fireEvent(alterTableCompactionExceptionEvent, operationContext)
         compactionException = operationContext.getProperty("compactionException").toString
     }
-
     if (compactionException.equalsIgnoreCase("true") && null == compactionType) {
       throw new MalformedCarbonCommandException(
         "Unsupported alter operation on carbon table")
@@ -159,8 +168,7 @@ case class CarbonAlterTableCompactionCommand(
       operationContext: OperationContext): Unit = {
     val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
     val compactionType = CompactionType.valueOf(alterTableModel.compactionType.toUpperCase)
-    val compactionSize: Long = CarbonDataMergerUtil
-      .getCompactionSize(compactionType, carbonLoadModel)
+    val compactionSize = CarbonDataMergerUtil.getCompactionSize(compactionType, carbonLoadModel)
     if (CompactionType.IUD_UPDDEL_DELTA == compactionType) {
       if (alterTableModel.segmentUpdateStatusManager.isDefined) {
         carbonLoadModel.setSegmentUpdateStatusManager(
@@ -175,7 +183,7 @@ case class CarbonAlterTableCompactionCommand(
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
 
     if (null == carbonLoadModel.getLoadMetadataDetails) {
-      CommonUtil.readLoadMetadataDetails(carbonLoadModel)
+      carbonLoadModel.readAndSetLoadMetadataDetails()
     }
 
     if (compactionType == CompactionType.STREAMING) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 5dbd383..4806a6f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -46,6 +46,7 @@ import org.apache.spark.storage.StorageLevel
 import org.apache.spark.unsafe.types.UTF8String
 import org.apache.spark.util.{CarbonReflectionUtils, CausedBy, FileUtils}
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
 import org.apache.carbondata.core.datamap.DataMapStoreManager
@@ -70,12 +71,15 @@ import org.apache.carbondata.processing.exception.DataLoadingException
 import org.apache.carbondata.processing.loading.TableProcessingOperations
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadMetadataEvent, LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
+import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
 import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, CarbonDropPartitionRDD}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, GlobalDictionaryUtil}
 import org.apache.carbondata.spark.load.DataLoadProcessorStepOnSpark
 import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataLoadingUtil, GlobalDictionaryUtil, SparkDataTypeConverterImpl}
@@ -174,7 +178,7 @@ case class CarbonLoadDataCommand(
     val carbonLoadModel = new CarbonLoadModel()
     try {
       val tableProperties = table.getTableInfo.getFactTable.getTableProperties
-      val optionsFinal = DataLoadingUtil.getDataLoadingOptions(carbonProperty, options)
+      val optionsFinal = LoadOption.fillOptionWithDefaultValue(options.asJava)
       optionsFinal.put("sort_scope", tableProperties.asScala.getOrElse("sort_scope",
         carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
           carbonProperty.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
@@ -189,10 +193,8 @@ case class CarbonLoadDataCommand(
       carbonLoadModel.setAggLoadRequest(
         internalOptions.getOrElse(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL, "false").toBoolean)
       carbonLoadModel.setSegmentId(internalOptions.getOrElse("mergedSegmentName", ""))
-      DataLoadingUtil.buildCarbonLoadModel(
-        table,
-        carbonProperty,
-        options,
+      new CarbonLoadModelBuilder(table).build(
+        options.asJava,
         optionsFinal,
         carbonLoadModel,
         hadoopConf,
@@ -221,7 +223,7 @@ case class CarbonLoadDataCommand(
             carbonLoadModel,
             factPath,
             dataFrame.isDefined,
-            optionsFinal.asJava,
+            optionsFinal,
             options.asJava,
             isOverwriteTable)
         operationContext.setProperty("isOverwrite", isOverwriteTable)
@@ -229,6 +231,7 @@ case class CarbonLoadDataCommand(
         // First system has to partition the data first and then call the load data
         LOGGER.info(s"Initiating Direct Load for the Table : ($dbName.$tableName)")
         // Clean up the old invalid segment data before creating a new entry for new load.
+        SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, false)
         DataLoadingUtil.deleteLoadsAndUpdateMetadata(
           isForceDeletion = false,
           table,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
index a37d6dc..f074285 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
@@ -22,13 +22,13 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 
+import org.apache.carbondata.common.exceptions.ConcurrentOperationException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.events.{DeleteFromTablePostEvent, DeleteFromTablePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.loading.FailureCauses
-import org.apache.carbondata.spark.exception.ConcurrentOperationException
 
 /**
  * IUD update delete and compaction framework.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
index 4886676..5165342 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
@@ -25,6 +25,7 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.types.ArrayType
 import org.apache.spark.storage.StorageLevel
 
+import org.apache.carbondata.common.exceptions.ConcurrentOperationException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.Segment
@@ -34,7 +35,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus, UpdateTablePostEvent, UpdateTablePreEvent}
 import org.apache.carbondata.processing.loading.FailureCauses
-import org.apache.carbondata.spark.exception.ConcurrentOperationException
 
 private[sql] case class CarbonProjectForUpdateCommand(
     plan: LogicalPlan,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
index 5817d88..220d75d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
@@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan}
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.hive.HiveSessionCatalog
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 /**
  * Util for IUD common function

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index 4d0a4c5..c836584 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -35,7 +35,6 @@ import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider
 import org.apache.carbondata.core.metadata.schema.table.AggregationDataMapSchema
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
-import org.apache.carbondata.spark.util.DataLoadingUtil
 
 /**
  * Below command class will be used to create pre-aggregate table
@@ -179,11 +178,7 @@ case class CreatePreAggregateTableCommand(
     // This will be used to check if the parent table has any segments or not. If not then no
     // need to fire load for pre-aggregate table. Therefore reading the load details for PARENT
     // table.
-    DataLoadingUtil.deleteLoadsAndUpdateMetadata(isForceDeletion = false,
-      parentTable,
-      CarbonFilters.getCurrentPartitions(sparkSession,
-      TableIdentifier(parentTable.getTableName,
-        Some(parentTable.getDatabaseName))).map(_.asJava).orNull)
+    SegmentStatusManager.deleteLoadsAndUpdateMetadata(parentTable, false)
     val loadAvailable = SegmentStatusManager.readLoadMetadata(parentTable.getMetadataPath)
     if (loadAvailable.exists(load => load.getSegmentStatus == SegmentStatus.INSERT_IN_PROGRESS ||
       load.getSegmentStatus == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index 9df3241..adce70e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -34,6 +34,7 @@ import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 import org.apache.spark.sql.types.DataType
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
@@ -42,7 +43,6 @@ import org.apache.carbondata.core.metadata.schema.table.{AggregationDataMapSchem
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.format.TableInfo
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CommonUtil
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index 870c140..5f8eb12 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -23,6 +23,8 @@ import org.apache.spark.sql.execution.command.{AlterTableRenameModel, MetadataCo
 import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionCatalog}
 import org.apache.spark.util.AlterTableUtil
 
+import org.apache.carbondata.common.exceptions.ConcurrentOperationException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.DataMapStoreManager
@@ -35,7 +37,6 @@ import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{AlterTableRenamePostEvent, AlterTableRenamePreEvent, OperationContext, OperationListenerBus}
 import org.apache.carbondata.format.SchemaEvolutionEntry
-import org.apache.carbondata.spark.exception.{ConcurrentOperationException, MalformedCarbonCommandException}
 
 private[sql] case class CarbonAlterTableRenameCommand(
     alterTableRenameModel: AlterTableRenameModel)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
index 45767da..2b35416 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
@@ -18,12 +18,12 @@ package org.apache.spark.sql.execution.command.timeseries
 
 import org.apache.spark.sql.execution.command.{DataMapField, Field}
 
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES
 import org.apache.carbondata.core.metadata.schema.datamap.Granularity
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.preagg.TimeSeriesUDF
-import org.apache.carbondata.spark.exception.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 
 /**
  * Utility class for time series to keep

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
index b4d3bea..61a31a5 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
@@ -50,6 +50,9 @@ import org.apache.carbondata.hadoop.api.{CarbonOutputCommitter, CarbonTableOutpu
 import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat.CarbonRecordWriter
 import org.apache.carbondata.hadoop.internal.ObjectArrayWritable
 import org.apache.carbondata.hadoop.util.ObjectSerializationUtil
+import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable
+import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, Util}
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataLoadingUtil, Util}
 
@@ -87,7 +90,7 @@ with Serializable {
       TableIdentifier(options("tableName"), options.get("dbName")))(sparkSession)
     val model = new CarbonLoadModel
     val carbonProperty = CarbonProperties.getInstance()
-    val optionsFinal = DataLoadingUtil.getDataLoadingOptions(carbonProperty, options)
+    val optionsFinal = LoadOption.fillOptionWithDefaultValue(options.asJava)
     val tableProperties = table.getTableInfo.getFactTable.getTableProperties
     optionsFinal.put("sort_scope", tableProperties.asScala.getOrElse("sort_scope",
       carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
@@ -102,14 +105,11 @@ with Serializable {
     val optionsLocal = new mutable.HashMap[String, String]()
     optionsLocal ++= options
     optionsLocal += (("header", "false"))
-    DataLoadingUtil.buildCarbonLoadModel(
-      table,
-      carbonProperty,
-      optionsLocal.toMap,
+    new CarbonLoadModelBuilder(table).build(
+      optionsLocal.toMap.asJava,
       optionsFinal,
       model,
-      conf
-    )
+      conf)
     model.setUseOnePass(options.getOrElse("onepass", "false").toBoolean)
     model.setDictionaryServerHost(options.getOrElse("dicthost", null))
     model.setDictionaryServerPort(options.getOrElse("dictport", "-1").toInt)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
index f69ccc1..59ff58e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -32,10 +32,10 @@ import org.apache.spark.sql.execution.datasources.RefreshTable
 import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.util.{CarbonReflectionUtils, FileUtils}
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.processing.merger.CompactionType
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 /**
  * Carbon strategies for ddl commands

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
index 608ec60..7028dcf 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/StreamingTableStrategy.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.command.AlterTableRenameCommand
 import org.apache.spark.sql.execution.command.mutation.{CarbonProjectForDeleteCommand, CarbonProjectForUpdateCommand}
 import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableDataTypeChangeCommand, CarbonAlterTableDropColumnCommand}
 
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 
 /**
  * Strategy for streaming table, like blocking unsupported operation

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
index 7ca34af..27c7d17 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
@@ -24,9 +24,9 @@ import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.command.table.CarbonDropTableCommand
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, SessionParams}
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
   extends RunnableCommand {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
index 7addd26..a0f80f0 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
@@ -36,9 +36,9 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.CarbonReflectionUtils
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.spark.CarbonOption
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CommonUtil
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index ad6d0c7..ef4836e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -32,10 +32,10 @@ import org.apache.spark.sql.types.StructField
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.CarbonReflectionUtils
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.hadoop.util.SchemaReader
 import org.apache.carbondata.spark.CarbonOption
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CommonUtil
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index bc36e9c..aaa87a3 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionCatalog, HiveExternalCatalog}
 import org.apache.spark.sql.hive.HiveExternalCatalog._
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -39,7 +40,6 @@ import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.util.path.CarbonTablePath.getNewTablePath
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 object AlterTableUtil {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala
index bc62902..a8094b6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/TableAPIUtil.scala
@@ -19,10 +19,10 @@ package org.apache.spark.util
 
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 /**
  * table api util

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
index b9425d6..1e4d001 100644
--- a/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/spark2.2/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -44,10 +44,11 @@ import org.apache.spark.sql.parser.{CarbonHelperSqlAstBuilder, CarbonSpark2SqlPa
 import org.apache.spark.sql.types.DecimalType
 import org.apache.spark.util.CarbonReflectionUtils
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala
index c676b01..5ade510 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/segmentreading/TestSegmentReading.scala
@@ -4,7 +4,7 @@ import org.apache.spark.sql.Row
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 
 /**
  * Created by rahul on 19/9/17.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
index 1d41ddc..3298009 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
@@ -25,7 +25,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
 import org.apache.carbondata.processing.util.TableOptionConstant
 
 /**
@@ -63,7 +63,7 @@ class AllDictionaryTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
       CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
     carbonLoadModel.setCsvHeaderColumns(
-      CommonUtil.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
+      LoadOption.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
     // Create table and metadata folders if not exist
     val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)
     val fileType = FileFactory.getFileType(metadataDirectoryPath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index 7ca0b56..43d8c03 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -23,14 +23,14 @@ import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.exception.DataLoadingException
-import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
 import org.apache.carbondata.processing.util.TableOptionConstant
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 /**
  * test case for external column dictionary generation
@@ -176,7 +176,7 @@ class ExternalColumnDictionaryTestCase extends Spark2QueryTest with BeforeAndAft
       CarbonCommonConstants.CARBON_DATE_FORMAT,
       CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
     carbonLoadModel.setCsvHeaderColumns(
-      CommonUtil.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
+      LoadOption.getCsvHeaderColumns(carbonLoadModel, FileFactory.getConfiguration))
     carbonLoadModel.setMaxColumns("100")
     // Create table and metadata folders if not exist
     val metadataDirectoryPath = CarbonTablePath.getMetadataPath(table.getTablePath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index f1c27ae..9d9f01d 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -32,14 +32,12 @@ import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery}
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{FileFormat, SegmentStatus}
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.spark.exception.{MalformedCarbonCommandException, ProcessMetaDataException}
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.exception.ProcessMetaDataException
 
 class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
index 9da7244..65a006b 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
@@ -21,11 +21,11 @@ import org.apache.spark.sql.common.util.Spark2QueryTest
 import org.apache.spark.sql.execution.exchange.ShuffleExchange
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
index ac10b9a..995f041 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AddColumnTestCases.scala
@@ -26,9 +26,9 @@ import org.apache.spark.sql.common.util.Spark2QueryTest
 import org.apache.spark.sql.test.TestQueryExecutor
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.{MalformedCarbonCommandException, ProcessMetaDataException}
 
 class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index 638ad39..890492e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -18,14 +18,16 @@
 package org.apache.carbondata.processing.loading.model;
 
 import java.io.Serializable;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 
 import org.apache.carbondata.core.dictionary.service.DictionaryServiceProvider;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
-
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 public class CarbonLoadModel implements Serializable {
 
@@ -868,6 +870,7 @@ public class CarbonLoadModel implements Serializable {
     this.skipEmptyLine = skipEmptyLine;
   }
 
+
   public boolean isPartitionLoad() {
     return isPartitionLoad;
   }
@@ -883,4 +886,13 @@ public class CarbonLoadModel implements Serializable {
   public void setDataWritePath(String dataWritePath) {
     this.dataWritePath = dataWritePath;
   }
+
+  /**
+   * Read segments metadata from table status file and set it to this load model object
+   */
+  public void readAndSetLoadMetadataDetails() {
+    String metadataPath = CarbonTablePath.getMetadataPath(tablePath);
+    LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metadataPath);
+    setLoadMetadataDetails(Arrays.asList(details));
+  }
 }


[25/49] carbondata git commit: [CARBONDATA-1544][Datamap] Datamap FineGrain implementation

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
index 78544d3..fe0bbcf 100644
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataWriter.java
@@ -19,7 +19,6 @@ package org.apache.carbondata.datamap.examples;
 
 import java.io.BufferedWriter;
 import java.io.DataOutputStream;
-import java.io.File;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.util.ArrayList;
@@ -29,17 +28,18 @@ import java.util.Map;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import com.google.gson.Gson;
 
-public class MinMaxDataWriter implements DataMapWriter {
+public class MinMaxDataWriter extends AbstractDataMapWriter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(TableInfo.class.getName());
@@ -50,17 +50,23 @@ public class MinMaxDataWriter implements DataMapWriter {
 
   private Map<Integer, BlockletMinMax> blockMinMaxMap;
 
-  private String blockPath;
+  private String dataWritePath;
 
+  public MinMaxDataWriter(AbsoluteTableIdentifier identifier, String segmentId,
+      String dataWritePath) {
+    super(identifier, segmentId, dataWritePath);
+    this.identifier = identifier;
+    this.segmentId = segmentId;
+    this.dataWritePath = dataWritePath;
+  }
 
-  @Override public void onBlockStart(String blockId, String blockPath) {
+  @Override public void onBlockStart(String blockId) {
     pageLevelMax = null;
     pageLevelMin = null;
     blockletLevelMax = null;
     blockletLevelMin = null;
     blockMinMaxMap = null;
     blockMinMaxMap = new HashMap<Integer, BlockletMinMax>();
-    this.blockPath = blockPath;
   }
 
   @Override public void onBlockEnd(String blockId) {
@@ -161,7 +167,7 @@ public class MinMaxDataWriter implements DataMapWriter {
     List<MinMaxIndexBlockDetails> tempMinMaxIndexBlockDetails = null;
     tempMinMaxIndexBlockDetails = loadBlockDetails();
     try {
-      writeMinMaxIndexFile(tempMinMaxIndexBlockDetails, blockPath, blockId);
+      writeMinMaxIndexFile(tempMinMaxIndexBlockDetails, blockId);
     } catch (IOException ex) {
       LOGGER.info(" Unable to write the file");
     }
@@ -178,7 +184,6 @@ public class MinMaxDataWriter implements DataMapWriter {
       tmpminMaxIndexBlockDetails.setMinValues(blockMinMaxMap.get(index).getMin());
       tmpminMaxIndexBlockDetails.setMaxValues(blockMinMaxMap.get(index).getMax());
       tmpminMaxIndexBlockDetails.setBlockletId(index);
-      tmpminMaxIndexBlockDetails.setFilePath(this.blockPath);
       minMaxIndexBlockDetails.add(tmpminMaxIndexBlockDetails);
     }
     return minMaxIndexBlockDetails;
@@ -187,22 +192,19 @@ public class MinMaxDataWriter implements DataMapWriter {
   /**
    * Write the data to a file. This is JSON format file.
    * @param minMaxIndexBlockDetails
-   * @param blockPath
    * @param blockId
    * @throws IOException
    */
   public void writeMinMaxIndexFile(List<MinMaxIndexBlockDetails> minMaxIndexBlockDetails,
-      String blockPath, String blockId) throws IOException {
-    String filePath = blockPath.substring(0, blockPath.lastIndexOf(File.separator) + 1) + blockId
-        + ".minmaxindex";
+      String blockId) throws IOException {
+    String filePath = dataWritePath +"/" + blockId + ".minmaxindex";
     BufferedWriter brWriter = null;
     DataOutputStream dataOutStream = null;
     try {
       FileFactory.createNewFile(filePath, FileFactory.getFileType(filePath));
       dataOutStream = FileFactory.getDataOutputStream(filePath, FileFactory.getFileType(filePath));
       Gson gsonObjectToWrite = new Gson();
-      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutStream,
-          CarbonCommonConstants.CARBON_DEFAULT_STREAM_ENCODEFORMAT));
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutStream, "UTF-8"));
       String minmaxIndexData = gsonObjectToWrite.toJson(minMaxIndexBlockDetails);
       brWriter.write(minmaxIndexData);
     } catch (IOException ioe) {
@@ -215,7 +217,11 @@ public class MinMaxDataWriter implements DataMapWriter {
         dataOutStream.flush();
       }
       CarbonUtil.closeStreams(brWriter, dataOutStream);
+      commitFile(filePath);
     }
   }
 
+  @Override public void finish() throws IOException {
+
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
index 0596db5..93a453e 100644
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxIndexBlockDetails.java
@@ -33,11 +33,6 @@ public class MinMaxIndexBlockDetails implements Serializable {
   private byte[][] maxValues;
 
   /**
-   * filePath pointing to the block.
-   */
-  private String filePath;
-
-  /**
    * BlockletID of the block.
    */
   private Integer BlockletId;
@@ -59,14 +54,6 @@ public class MinMaxIndexBlockDetails implements Serializable {
     this.maxValues = maxValues;
   }
 
-  public String getFilePath() {
-    return filePath;
-  }
-
-  public void setFilePath(String filePath) {
-    this.filePath = filePath;
-  }
-
   public Integer getBlockletId() {
     return BlockletId;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
index a4c6e4a..c586f3c 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java
@@ -87,6 +87,8 @@ public class CarbonInputSplit extends FileSplit
 
   private FileFormat fileFormat = FileFormat.COLUMNAR_V3;
 
+  private String dataMapWritePath;
+
   public CarbonInputSplit() {
     segmentId = null;
     taskId = "0";
@@ -98,7 +100,8 @@ public class CarbonInputSplit extends FileSplit
   }
 
   private CarbonInputSplit(String segmentId, String blockletId, Path path, long start, long length,
-      String[] locations, ColumnarFormatVersion version, String[] deleteDeltaFiles) {
+      String[] locations, ColumnarFormatVersion version, String[] deleteDeltaFiles,
+      String dataMapWritePath) {
     super(path, start, length, locations);
     this.segmentId = segmentId;
     String taskNo = CarbonTablePath.DataFileUtil.getTaskNo(path.getName());
@@ -111,12 +114,13 @@ public class CarbonInputSplit extends FileSplit
     this.invalidSegments = new ArrayList<>();
     this.version = version;
     this.deleteDeltaFiles = deleteDeltaFiles;
+    this.dataMapWritePath = dataMapWritePath;
   }
 
   public CarbonInputSplit(String segmentId, String blockletId, Path path, long start, long length,
       String[] locations, int numberOfBlocklets, ColumnarFormatVersion version,
       String[] deleteDeltaFiles) {
-    this(segmentId, blockletId, path, start, length, locations, version, deleteDeltaFiles);
+    this(segmentId, blockletId, path, start, length, locations, version, deleteDeltaFiles, null);
     this.numberOfBlocklets = numberOfBlocklets;
   }
 
@@ -166,9 +170,9 @@ public class CarbonInputSplit extends FileSplit
   }
 
   public static CarbonInputSplit from(String segmentId, String blockletId, FileSplit split,
-      ColumnarFormatVersion version) throws IOException {
+      ColumnarFormatVersion version, String dataMapWritePath) throws IOException {
     return new CarbonInputSplit(segmentId, blockletId, split.getPath(), split.getStart(),
-        split.getLength(), split.getLocations(), version, null);
+        split.getLength(), split.getLocations(), version, null, dataMapWritePath);
   }
 
   public static List<TableBlockInfo> createBlocks(List<CarbonInputSplit> splitList) {
@@ -182,6 +186,7 @@ public class CarbonInputSplit extends FileSplit
                 split.getSegmentId(), split.getLocations(), split.getLength(), blockletInfos,
                 split.getVersion(), split.getDeleteDeltaFiles());
         blockInfo.setDetailInfo(split.getDetailInfo());
+        blockInfo.setDataMapWriterPath(split.dataMapWritePath);
         blockInfo.setBlockOffset(split.getDetailInfo().getBlockFooterOffset());
         tableBlockInfoList.add(blockInfo);
       } catch (IOException e) {
@@ -233,6 +238,10 @@ public class CarbonInputSplit extends FileSplit
       detailInfo = new BlockletDetailInfo();
       detailInfo.readFields(in);
     }
+    boolean dataMapWriterPathExists = in.readBoolean();
+    if (dataMapWriterPathExists) {
+      dataMapWritePath = in.readUTF();
+    }
   }
 
   @Override public void write(DataOutput out) throws IOException {
@@ -255,6 +264,10 @@ public class CarbonInputSplit extends FileSplit
     if (detailInfo != null) {
       detailInfo.write(out);
     }
+    out.writeBoolean(dataMapWritePath != null);
+    if (dataMapWritePath != null) {
+      out.writeUTF(dataMapWritePath);
+    }
   }
 
   public List<String> getInvalidSegments() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 9dc8d38..b485b69 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -34,6 +34,7 @@ import java.util.Map;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.datamap.TableDataMap;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -735,16 +736,17 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     // get tokens for all the required FileSystem for table path
     TokenCache.obtainTokensForNamenodes(job.getCredentials(),
         new Path[] { new Path(absoluteTableIdentifier.getTablePath()) }, job.getConfiguration());
-
-    TableDataMap blockletMap = DataMapStoreManager.getInstance()
-        .getDataMap(absoluteTableIdentifier, BlockletDataMap.NAME,
-            BlockletDataMapFactory.class.getName());
+    boolean distributedCG = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
+            CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT));
+    TableDataMap blockletMap =
+        DataMapStoreManager.getInstance().chooseDataMap(absoluteTableIdentifier);
     DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
     List<PartitionSpec> partitionsToPrune = getPartitionsToPrune(job.getConfiguration());
     List<ExtendedBlocklet> prunedBlocklets;
-    if (dataMapJob != null) {
+    if (distributedCG || blockletMap.getDataMapFactory().getDataMapType() == DataMapType.FG) {
       DistributableDataMapFormat datamapDstr =
-          new DistributableDataMapFormat(absoluteTableIdentifier, BlockletDataMap.NAME,
+          new DistributableDataMapFormat(absoluteTableIdentifier, blockletMap.getDataMapName(),
               segmentIds, partitionsToPrune,
               BlockletDataMapFactory.class.getName());
       prunedBlocklets = dataMapJob.execute(datamapDstr, resolver);
@@ -797,7 +799,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
         org.apache.carbondata.hadoop.CarbonInputSplit.from(blocklet.getSegmentId(),
             blocklet.getBlockletId(), new FileSplit(new Path(blocklet.getPath()), 0,
                 blocklet.getLength(), blocklet.getLocations()),
-            ColumnarFormatVersion.valueOf((short) blocklet.getDetailInfo().getVersionNumber()));
+            ColumnarFormatVersion.valueOf((short) blocklet.getDetailInfo().getVersionNumber()),
+            blocklet.getDataMapWriterPath());
     split.setDetailInfo(blocklet.getDetailInfo());
     return split;
   }

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index c80ee2b..2f8a1d1 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -20,25 +20,32 @@ package org.apache.carbondata.spark.testsuite.datamap
 import java.util
 
 import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.{DataFrame, SaveMode}
 import org.apache.spark.sql.test.util.QueryTest
+import org.apache.spark.sql.{DataFrame, SaveMode}
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datamap.dev.{DataMap, DataMapFactory, DataMapWriter}
 import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter
+import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainDataMap, AbstractCoarseGrainDataMapFactory}
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
 import org.apache.carbondata.core.datastore.page.ColumnPage
-import org.apache.carbondata.core.indexstore.schema.FilterType
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType
+import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.events.Event
 
-class C2DataMapFactory() extends DataMapFactory {
+class C2DataMapFactory() extends AbstractCoarseGrainDataMapFactory {
+
+  var identifier: AbsoluteTableIdentifier = _
 
   override def init(identifier: AbsoluteTableIdentifier,
-      dataMapName: String): Unit = {}
+      dataMapName: String): Unit = {
+    this.identifier = identifier
+  }
 
   override def fireEvent(event: Event): Unit = ???
 
@@ -46,13 +53,13 @@ class C2DataMapFactory() extends DataMapFactory {
 
   override def clear(): Unit = {}
 
-  override def getDataMaps(distributable: DataMapDistributable): java.util.List[DataMap] = ???
+  override def getDataMaps(distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainDataMap] = ???
 
   override def getDataMaps(segmentId: Segment): util.List[DataMap] = ???
 
-  override def createWriter(segmentId: Segment): DataMapWriter = DataMapWriterSuite.dataMapWriterC2Mock
+  override def createWriter(segmentId: Segment): AbstractDataMapWriter = DataMapWriterSuite.dataMapWriterC2Mock
 
-  override def getMeta: DataMapMeta = new DataMapMeta(List("c2").asJava, FilterType.EQUALTO)
+  override def getMeta: DataMapMeta = new DataMapMeta(List("c2").asJava, List(ExpressionType.EQUALS).asJava)
 
   /**
    * Get all distributable objects of a segmentid
@@ -62,6 +69,7 @@ class C2DataMapFactory() extends DataMapFactory {
   override def toDistributable(segmentId: Segment): util.List[DataMapDistributable] = {
     ???
   }
+
 }
 
 class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
@@ -164,9 +172,12 @@ class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
 }
 
 object DataMapWriterSuite {
+
   var callbackSeq: Seq[String] = Seq[String]()
 
-  val dataMapWriterC2Mock = new DataMapWriter {
+  def dataMapWriterC2Mock(identifier: AbsoluteTableIdentifier, segmentId: String,
+      dataWritePath: String) =
+    new AbstractDataMapWriter(identifier, segmentId, dataWritePath) {
 
     override def onPageAdded(
         blockletId: Int,
@@ -191,9 +202,21 @@ object DataMapWriterSuite {
       callbackSeq :+= s"blocklet start $blockletId"
     }
 
-    override def onBlockStart(blockId: String, blockPath: String): Unit = {
+    /**
+     * Start of new block notification.
+     *
+     * @param blockId file name of the carbondata file
+     */
+    override def onBlockStart(blockId: String) = {
       callbackSeq :+= s"block start $blockId"
     }
 
+    /**
+     * This is called during closing of writer.So after this call no more data will be sent to this
+     * class.
+     */
+    override def finish() = {
+
+    }
   }
 }
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
index f2cdd67..5550358 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
@@ -28,11 +28,14 @@ import org.apache.spark.sql.{DataFrame, SaveMode}
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.dev.{DataMap, DataMapFactory, DataMapWriter}
+import org.apache.carbondata.core.datamap.dev.{DataMap, DataMapFactory}
 import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
+import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainDataMap, AbstractCoarseGrainDataMapFactory}
+import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMap}
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
 import org.apache.carbondata.core.datastore.page.ColumnPage
-import org.apache.carbondata.core.indexstore.schema.FilterType
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.events.Event
 import org.apache.carbondata.spark.exception.ConcurrentOperationException
@@ -264,7 +267,7 @@ object Global {
   var overwriteRunning = false
 }
 
-class WaitingDataMap() extends DataMapFactory {
+class WaitingDataMap() extends AbstractCoarseGrainDataMapFactory {
 
   override def init(identifier: AbsoluteTableIdentifier, dataMapName: String): Unit = { }
 
@@ -274,12 +277,12 @@ class WaitingDataMap() extends DataMapFactory {
 
   override def clear(): Unit = {}
 
-  override def getDataMaps(distributable: DataMapDistributable): java.util.List[DataMap] = ???
+  override def getDataMaps(distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainDataMap] = ???
 
   override def getDataMaps(segmentId: Segment): util.List[DataMap] = ???
 
-  override def createWriter(segmentId: Segment): DataMapWriter = {
-    new DataMapWriter {
+  override def createWriter(segmentId: Segment): AbstractDataMapWriter = {
+    new AbstractDataMapWriter {
       override def onPageAdded(blockletId: Int, pageId: Int, pages: Array[ColumnPage]): Unit = { }
 
       override def onBlockletEnd(blockletId: Int): Unit = { }
@@ -295,10 +298,14 @@ class WaitingDataMap() extends DataMapFactory {
         // wait for 1 second to let second SQL to finish
         Thread.sleep(1000)
       }
+
+      override def finish(): Unit = {
+
+      }
     }
   }
 
-  override def getMeta: DataMapMeta = new DataMapMeta(List("o_country").asJava, FilterType.EQUALTO)
+  override def getMeta: DataMapMeta = new DataMapMeta(List("o_country").asJava, Seq(ExpressionType.EQUALS).asJava)
 
   override def toDistributable(segmentId: Segment): util.List[DataMapDistributable] = ???
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 97be1fb..ddb9b32 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -448,9 +448,10 @@ class CarbonScanRDD(
     CarbonTableInputFormat.setQuerySegment(conf, identifier)
     CarbonTableInputFormat.setFilterPredicates(conf, filterExpression)
     CarbonTableInputFormat.setColumnProjection(conf, columnProjection)
-    if (CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
-        CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT).toBoolean) {
+    CarbonTableInputFormat.setDataMapJob(conf, new SparkDataMapJob)
+    if (CarbonProperties.getInstance().getProperty(
+      CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP,
+      CarbonCommonConstants.USE_DISTRIBUTED_DATAMAP_DEFAULT).toBoolean) {
       CarbonTableInputFormat.setDataMapJob(conf, new SparkDataMapJob)
     }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
index 017bca8..f1c27ae 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/TestStreamingTableOperation.scala
@@ -1232,8 +1232,9 @@ class TestStreamingTableOperation extends QueryTest with BeforeAndAfterAll {
             .start()
           qry.awaitTermination()
         } catch {
-          case ex =>
-            throw new Exception(ex.getMessage)
+          case ex: Throwable =>
+            LOGGER.error(ex.getMessage)
+            throw new Exception(ex.getMessage, ex)
         } finally {
           if (null != qry) {
             qry.stop()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
index 6fbbd3e..5083ab5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.processing.datamap;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
@@ -29,8 +30,8 @@ import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datamap.TableDataMap;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.processing.store.TablePage;
@@ -44,25 +45,26 @@ public class DataMapWriterListener {
       DataMapWriterListener.class.getCanonicalName());
 
   // list indexed column name -> list of data map writer
-  private Map<List<String>, List<DataMapWriter>> registry = new ConcurrentHashMap<>();
+  private Map<List<String>, List<AbstractDataMapWriter>> registry = new ConcurrentHashMap<>();
 
   /**
    * register all datamap writer for specified table and segment
    */
-  public void registerAllWriter(AbsoluteTableIdentifier identifier, String segmentId) {
+  public void registerAllWriter(AbsoluteTableIdentifier identifier, String segmentId,
+      String dataWritePath) {
     List<TableDataMap> tableDataMaps = DataMapStoreManager.getInstance().getAllDataMap(identifier);
     if (tableDataMaps != null) {
       for (TableDataMap tableDataMap : tableDataMaps) {
         DataMapFactory factory = tableDataMap.getDataMapFactory();
-        register(factory, segmentId);
+        register(factory, segmentId, dataWritePath);
       }
     }
   }
 
   /**
-   * Register a DataMapWriter
+   * Register a AbstractDataMapWriter
    */
-  private void register(DataMapFactory factory, String segmentId) {
+  private void register(DataMapFactory factory, String segmentId, String dataWritePath) {
     assert (factory != null);
     assert (segmentId != null);
     DataMapMeta meta = factory.getMeta();
@@ -71,8 +73,8 @@ public class DataMapWriterListener {
       return;
     }
     List<String> columns = factory.getMeta().getIndexedColumns();
-    List<DataMapWriter> writers = registry.get(columns);
-    DataMapWriter writer = factory.createWriter(new Segment(segmentId, null));
+    List<AbstractDataMapWriter> writers = registry.get(columns);
+    AbstractDataMapWriter writer = factory.createWriter(new Segment(segmentId, null));
     if (writers != null) {
       writers.add(writer);
     } else {
@@ -80,36 +82,36 @@ public class DataMapWriterListener {
       writers.add(writer);
       registry.put(columns, writers);
     }
-    LOG.info("DataMapWriter " + writer + " added");
+    LOG.info("AbstractDataMapWriter " + writer + " added");
   }
 
   public void onBlockStart(String blockId, String blockPath) {
-    for (List<DataMapWriter> writers : registry.values()) {
-      for (DataMapWriter writer : writers) {
-        writer.onBlockStart(blockId, blockPath);
+    for (List<AbstractDataMapWriter> writers : registry.values()) {
+      for (AbstractDataMapWriter writer : writers) {
+        writer.onBlockStart(blockId);
       }
     }
   }
 
   public void onBlockEnd(String blockId) {
-    for (List<DataMapWriter> writers : registry.values()) {
-      for (DataMapWriter writer : writers) {
+    for (List<AbstractDataMapWriter> writers : registry.values()) {
+      for (AbstractDataMapWriter writer : writers) {
         writer.onBlockEnd(blockId);
       }
     }
   }
 
   public void onBlockletStart(int blockletId) {
-    for (List<DataMapWriter> writers : registry.values()) {
-      for (DataMapWriter writer : writers) {
+    for (List<AbstractDataMapWriter> writers : registry.values()) {
+      for (AbstractDataMapWriter writer : writers) {
         writer.onBlockletStart(blockletId);
       }
     }
   }
 
   public void onBlockletEnd(int blockletId) {
-    for (List<DataMapWriter> writers : registry.values()) {
-      for (DataMapWriter writer : writers) {
+    for (List<AbstractDataMapWriter> writers : registry.values()) {
+      for (AbstractDataMapWriter writer : writers) {
         writer.onBlockletEnd(blockletId);
       }
     }
@@ -122,18 +124,29 @@ public class DataMapWriterListener {
    * @param tablePage  page data
    */
   public void onPageAdded(int blockletId, int pageId, TablePage tablePage) {
-    Set<Map.Entry<List<String>, List<DataMapWriter>>> entries = registry.entrySet();
-    for (Map.Entry<List<String>, List<DataMapWriter>> entry : entries) {
+    Set<Map.Entry<List<String>, List<AbstractDataMapWriter>>> entries = registry.entrySet();
+    for (Map.Entry<List<String>, List<AbstractDataMapWriter>> entry : entries) {
       List<String> indexedColumns = entry.getKey();
       ColumnPage[] pages = new ColumnPage[indexedColumns.size()];
       for (int i = 0; i < indexedColumns.size(); i++) {
         pages[i] = tablePage.getColumnPage(indexedColumns.get(i));
       }
-      List<DataMapWriter> writers = entry.getValue();
-      for (DataMapWriter writer : writers) {
+      List<AbstractDataMapWriter> writers = entry.getValue();
+      for (AbstractDataMapWriter writer : writers) {
         writer.onPageAdded(blockletId, pageId, pages);
       }
     }
   }
 
+  /**
+   * Finish all datamap writers
+   */
+  public void finish() throws IOException {
+    for (List<AbstractDataMapWriter> writers : registry.values()) {
+      for (AbstractDataMapWriter writer : writers) {
+        writer.finish();
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index 5062a78..d6af747 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Random;
 
 import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
@@ -259,7 +260,8 @@ public class CarbonFactDataHandlerModel {
     carbonFactDataHandlerModel.sortScope = CarbonDataProcessorUtil.getSortScope(configuration);
 
     DataMapWriterListener listener = new DataMapWriterListener();
-    listener.registerAllWriter(configuration.getTableIdentifier(), configuration.getSegmentId());
+    listener.registerAllWriter(configuration.getTableIdentifier(), configuration.getSegmentId(),
+        storeLocation[new Random().nextInt(storeLocation.length)]);
     carbonFactDataHandlerModel.dataMapWriterlistener = listener;
     carbonFactDataHandlerModel.writingCoresCount = configuration.getWritingCoresCount();
 
@@ -322,6 +324,12 @@ public class CarbonFactDataHandlerModel {
         segmentProperties.getDimensions(),
         segmentProperties.getMeasures());
 
+    DataMapWriterListener listener = new DataMapWriterListener();
+    listener.registerAllWriter(
+        loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier(),
+        loadModel.getSegmentId(),
+        tempStoreLocation[new Random().nextInt(tempStoreLocation.length)]);
+    carbonFactDataHandlerModel.dataMapWriterlistener = listener;
     return carbonFactDataHandlerModel;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
index 02391cf..8d26ad2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -17,8 +17,6 @@
 
 package org.apache.carbondata.processing.store.writer;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileFilter;
 import java.io.FileNotFoundException;
@@ -41,14 +39,11 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
 import org.apache.carbondata.core.metadata.index.BlockIndexInfo;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonMergerUtil;
 import org.apache.carbondata.core.util.CarbonMetadataUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
@@ -63,7 +58,6 @@ import org.apache.carbondata.processing.datamap.DataMapWriterListener;
 import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
 
 import org.apache.commons.lang3.ArrayUtils;
-import org.apache.hadoop.io.IOUtils;
 
 public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
 
@@ -71,12 +65,6 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
       LogServiceFactory.getLogService(AbstractFactDataWriter.class.getName());
 
   /**
-   * dfs.bytes-per-checksum
-   * HDFS checksum length, block size for a file should be exactly divisible
-   * by this value
-   */
-  private static final int HDFS_CHECKSUM_LENGTH = 512;
-  /**
    * file channel
    */
   protected FileChannel fileChannel;
@@ -208,35 +196,6 @@ public abstract class AbstractFactDataWriter 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;
-    }
-    // convert to make block size more readable.
-    String readableBlockSize = ByteUtil.convertByteToReadable(blockSize);
-    String readableFileSize = ByteUtil.convertByteToReadable(fileSize);
-    String readableMaxSize = ByteUtil.convertByteToReadable(maxSize);
-    LOGGER.info(
-        "The configured block size is " + readableBlockSize + ", the actual carbon file size is "
-            + readableFileSize + ", choose the max value " + readableMaxSize
-            + " as the block size on HDFS");
-    return maxSize;
-  }
-
-  /**
    * This method will be used to update the file channel with new file if exceeding block size
    * threshold, new file will be created once existing file reached the file size limit This
    * method will first check whether existing file size is exceeded the file
@@ -282,7 +241,7 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
 
   private String constructFactFileFullPath() {
     String factFilePath =
-        this.dataWriterVo.getCarbonDataDirectoryPath() + File.separator + this.carbonDataFileName;
+        this.model.getCarbonDataDirectoryPath() + File.separator + this.carbonDataFileName;
     return factFilePath;
   }
   /**
@@ -293,7 +252,9 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     notifyDataMapBlockEnd();
     CarbonUtil.closeStreams(this.fileOutputStream, this.fileChannel);
     if (copyInCurrentThread) {
-      copyCarbonDataFileToCarbonStorePath(carbonDataFileTempPath);
+      CarbonUtil.copyCarbonDataFileToCarbonStorePath(
+          carbonDataFileTempPath, model.getCarbonDataDirectoryPath(),
+          fileSizeInBytes);
     } else {
       executorServiceSubmitList.add(executorService.submit(new CopyThread(carbonDataFileTempPath)));
     }
@@ -446,7 +407,9 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     }
     writer.close();
     // copy from temp to actual store location
-    copyCarbonDataFileToCarbonStorePath(fileName);
+    CarbonUtil.copyCarbonDataFileToCarbonStorePath(fileName,
+            model.getCarbonDataDirectoryPath(),
+            fileSizeInBytes);
   }
 
   /**
@@ -456,80 +419,20 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
    * @throws CarbonDataWriterException
    */
   protected void closeExecutorService() throws CarbonDataWriterException {
-    executorService.shutdown();
     try {
+      listener.finish();
+      executorService.shutdown();
       executorService.awaitTermination(2, TimeUnit.HOURS);
-    } catch (InterruptedException e) {
-      throw new CarbonDataWriterException(e.getMessage());
-    }
-    for (int i = 0; i < executorServiceSubmitList.size(); i++) {
-      try {
+      for (int i = 0; i < executorServiceSubmitList.size(); i++) {
         executorServiceSubmitList.get(i).get();
-      } catch (InterruptedException e) {
-        throw new CarbonDataWriterException(e.getMessage());
-      } catch (ExecutionException e) {
-        throw new CarbonDataWriterException(e.getMessage());
       }
+    } catch (InterruptedException | ExecutionException | IOException e) {
+      LOGGER.error(e, "Error while finishing writer");
+      throw new CarbonDataWriterException(e.getMessage());
     }
   }
 
 
-  /**
-   * This method will copy the given file to carbon store location
-   *
-   * @param localFileName local file name with full path
-   * @throws CarbonDataWriterException
-   */
-  protected void copyCarbonDataFileToCarbonStorePath(String localFileName)
-      throws CarbonDataWriterException {
-    long copyStartTime = System.currentTimeMillis();
-    LOGGER.info("Copying " + localFileName + " --> " + model.getCarbonDataDirectoryPath());
-    try {
-      CarbonFile localCarbonFile =
-          FileFactory.getCarbonFile(localFileName, FileFactory.getFileType(localFileName));
-      String carbonFilePath = model.getCarbonDataDirectoryPath() + localFileName
-          .substring(localFileName.lastIndexOf(File.separator));
-      copyLocalFileToCarbonStore(carbonFilePath, localFileName,
-          CarbonCommonConstants.BYTEBUFFER_SIZE,
-          getMaxOfBlockAndFileSize(fileSizeInBytes, localCarbonFile.getSize()));
-    } catch (IOException e) {
-      throw new CarbonDataWriterException(
-          "Problem while copying file from local store to carbon store", e);
-    }
-    LOGGER.info(
-        "Total copy time (ms) to copy file " + localFileName + " is " + (System.currentTimeMillis()
-            - copyStartTime));
-  }
-
-  /**
-   * This method will read the local carbon data file and write to carbon data file in HDFS
-   *
-   * @param carbonStoreFilePath
-   * @param localFilePath
-   * @param bufferSize
-   * @param blockSize
-   * @throws IOException
-   */
-  private void copyLocalFileToCarbonStore(String carbonStoreFilePath, String localFilePath,
-      int bufferSize, long blockSize) throws IOException {
-    DataOutputStream dataOutputStream = null;
-    DataInputStream dataInputStream = null;
-    try {
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("HDFS file block size for file: " + carbonStoreFilePath + " is " + blockSize
-            + " (bytes");
-      }
-      dataOutputStream = FileFactory
-          .getDataOutputStream(carbonStoreFilePath, FileFactory.getFileType(carbonStoreFilePath),
-              bufferSize, blockSize);
-      dataInputStream = FileFactory
-          .getDataInputStream(localFilePath, FileFactory.getFileType(localFilePath), bufferSize);
-      IOUtils.copyBytes(dataInputStream, dataOutputStream, bufferSize);
-    } finally {
-      CarbonUtil.closeStream(dataInputStream);
-      CarbonUtil.closeStream(dataOutputStream);
-    }
-  }
 
   /**
    * This method will copy the carbon data file from local store location to
@@ -554,7 +457,10 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
      * @throws Exception if unable to compute a result
      */
     @Override public Void call() throws Exception {
-      copyCarbonDataFileToCarbonStorePath(fileName);
+      CarbonUtil.copyCarbonDataFileToCarbonStorePath(
+          fileName,
+          model.getCarbonDataDirectoryPath(),
+          fileSizeInBytes);
       return null;
     }
 


[18/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
[CARBONDATA-2099] Refactor query scan process to improve readability

Unified concepts in scan process flow:

1.QueryModel contains all parameter for scan, it is created by API in CarbonTable. (In future, CarbonTable will be the entry point for various table operations)
2.Use term ColumnChunk to represent one column in one blocklet, and use ChunkIndex in reader to read specified column chunk
3.Use term ColumnPage to represent one page in one ColumnChunk
4.QueryColumn => ProjectionColumn, indicating it is for projection

This closes #1874


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

Branch: refs/heads/carbonstore-rebase4
Commit: 975725a4b67d107de1f475087e358af3e055869c
Parents: 7f50828
Author: Jacky Li <ja...@qq.com>
Authored: Tue Jan 30 21:24:04 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Mon Feb 26 23:57:33 2018 +0800

----------------------------------------------------------------------
 .../dictionary/AbstractDictionaryCache.java     |   3 +-
 .../cache/dictionary/DictionaryCacheLoader.java |   7 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  11 +-
 .../core/datastore/BTreeBuilderInfo.java        |   6 -
 .../carbondata/core/datastore/DataRefNode.java  |  81 +--
 .../carbondata/core/datastore/FileHolder.java   | 118 ----
 .../carbondata/core/datastore/FileReader.java   | 114 +++
 .../core/datastore/block/SegmentProperties.java |  50 +-
 .../chunk/DimensionColumnDataChunk.java         | 116 ---
 .../datastore/chunk/DimensionColumnPage.java    | 111 +++
 .../chunk/impl/AbstractDimensionColumnPage.java |  89 +++
 .../chunk/impl/AbstractDimensionDataChunk.java  |  95 ---
 .../impl/ColumnGroupDimensionColumnPage.java    | 194 ++++++
 .../impl/ColumnGroupDimensionDataChunk.java     | 194 ------
 .../chunk/impl/DimensionRawColumnChunk.java     |  46 +-
 .../impl/FixedLengthDimensionColumnPage.java    | 163 +++++
 .../impl/FixedLengthDimensionDataChunk.java     | 163 -----
 .../chunk/impl/MeasureRawColumnChunk.java       |  26 +-
 .../impl/VariableLengthDimensionColumnPage.java | 133 ++++
 .../impl/VariableLengthDimensionDataChunk.java  | 140 ----
 .../reader/DimensionColumnChunkReader.java      |  14 +-
 .../chunk/reader/MeasureColumnChunkReader.java  |  12 +-
 .../AbstractChunkReaderV2V3Format.java          |  34 +-
 ...mpressedDimensionChunkFileBasedReaderV1.java |  38 +-
 ...mpressedDimensionChunkFileBasedReaderV2.java |  30 +-
 ...essedDimChunkFileBasedPageLevelReaderV3.java |  11 +-
 ...mpressedDimensionChunkFileBasedReaderV3.java |  49 +-
 .../AbstractMeasureChunkReaderV2V3Format.java   |  42 +-
 ...CompressedMeasureChunkFileBasedReaderV1.java |  16 +-
 ...CompressedMeasureChunkFileBasedReaderV2.java |  24 +-
 ...CompressedMeasureChunkFileBasedReaderV3.java |  45 +-
 ...essedMsrChunkFileBasedPageLevelReaderV3.java |   8 +-
 .../chunk/store/ColumnPageWrapper.java          |  30 +-
 .../chunk/store/DimensionDataChunkStore.java    |   8 +-
 .../SafeFixedLengthDimensionDataChunkStore.java |   6 +-
 ...feVariableLengthDimensionDataChunkStore.java |   8 +-
 ...nsafeFixedLengthDimensionDataChunkStore.java |  10 +-
 ...afeVariableLengthDimesionDataChunkStore.java |  10 +-
 .../datastore/columnar/ColumnGroupModel.java    |  26 -
 .../core/datastore/impl/DFSFileHolderImpl.java  | 166 -----
 .../core/datastore/impl/DFSFileReaderImpl.java  | 155 ++++
 .../datastore/impl/DefaultFileTypeProvider.java |  16 +-
 .../core/datastore/impl/FileFactory.java        |   4 +-
 .../core/datastore/impl/FileHolderImpl.java     | 224 ------
 .../core/datastore/impl/FileReaderImpl.java     | 215 ++++++
 .../core/datastore/impl/FileTypeInerface.java   |   4 +-
 .../impl/btree/AbstractBTreeLeafNode.java       |  60 +-
 .../impl/btree/BTreeDataRefNodeFinder.java      |   6 +-
 .../datastore/impl/btree/BTreeNonLeafNode.java  |  52 +-
 .../impl/btree/BlockBTreeLeafNode.java          |   6 +-
 .../impl/btree/BlockletBTreeLeafNode.java       |  46 +-
 .../page/encoding/EncodingFactory.java          |   8 +-
 .../server/NonSecureDictionaryServer.java       |   1 -
 .../core/indexstore/BlockletDetailInfo.java     |   4 -
 .../blockletindex/BlockletDataRefNode.java      | 228 ++++++
 .../BlockletDataRefNodeWrapper.java             | 241 -------
 .../indexstore/blockletindex/IndexWrapper.java  |   2 +-
 .../blockletindex/SegmentIndexFileStore.java    |   7 +-
 .../core/memory/HeapMemoryAllocator.java        |   2 +-
 .../core/metadata/blocklet/SegmentInfo.java     |  19 -
 .../core/metadata/schema/table/CarbonTable.java | 130 +++-
 .../schema/table/RelationIdentifier.java        |  16 -
 .../core/metadata/schema/table/TableInfo.java   |   6 +-
 .../schema/table/column/CarbonColumn.java       |   2 +-
 .../schema/table/column/CarbonDimension.java    |  12 -
 .../core/mutate/CarbonUpdateUtil.java           |  20 +-
 .../core/mutate/DeleteDeltaBlockDetails.java    |   8 -
 .../core/mutate/DeleteDeltaBlockletDetails.java |   8 -
 .../data/BlockletDeleteDeltaCacheLoader.java    |  87 ---
 .../scan/collector/ScannedResultCollector.java  |  16 +-
 .../impl/AbstractScannedResultCollector.java    |  29 +-
 .../impl/DictionaryBasedResultCollector.java    |  53 +-
 .../DictionaryBasedVectorResultCollector.java   |  63 +-
 .../collector/impl/RawBasedResultCollector.java |  32 +-
 ...structureBasedDictionaryResultCollector.java |  17 +-
 .../RestructureBasedRawResultCollector.java     |  27 +-
 .../RestructureBasedVectorResultCollector.java  |  33 +-
 .../core/scan/complextypes/ArrayQueryType.java  |   4 +-
 .../scan/complextypes/ComplexQueryType.java     |  12 +-
 .../scan/complextypes/PrimitiveQueryType.java   |   4 +-
 .../core/scan/complextypes/StructQueryType.java |   4 +-
 .../executor/impl/AbstractQueryExecutor.java    | 154 ++--
 .../scan/executor/impl/DetailQueryExecutor.java |   6 +-
 .../scan/executor/infos/BlockExecutionInfo.java | 179 ++---
 .../core/scan/executor/util/QueryUtil.java      | 141 ++--
 .../scan/executor/util/RestructureUtil.java     |  70 +-
 .../core/scan/expression/ColumnExpression.java  |   8 -
 .../scan/expression/FilterModificationNode.java |  35 +-
 .../expression/RangeExpressionEvaluator.java    |  11 +-
 .../scan/filter/FilterExpressionProcessor.java  |  22 +-
 .../core/scan/filter/FilterProcessor.java       |   2 +-
 .../carbondata/core/scan/filter/FilterUtil.java |  60 +-
 .../core/scan/filter/GenericQueryType.java      |   4 +-
 .../filter/executer/AndFilterExecuterImpl.java  |  22 +-
 .../ExcludeColGroupFilterExecuterImpl.java      |  85 ---
 .../executer/ExcludeFilterExecuterImpl.java     | 131 ++--
 .../scan/filter/executer/FilterExecuter.java    |  10 +-
 .../ImplicitIncludeFilterExecutorImpl.java      |  23 +-
 .../IncludeColGroupFilterExecuterImpl.java      |  31 +-
 .../executer/IncludeFilterExecuterImpl.java     | 142 ++--
 .../filter/executer/OrFilterExecuterImpl.java   |  25 +-
 .../executer/RangeValueFilterExecuterImpl.java  | 224 +++---
 .../RestructureExcludeFilterExecutorImpl.java   |  21 +-
 .../RestructureIncludeFilterExecutorImpl.java   |  22 +-
 .../executer/RowLevelFilterExecuterImpl.java    | 151 ++--
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  | 121 ++--
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java | 116 +--
 ...velRangeLessThanEqualFilterExecuterImpl.java | 113 +--
 .../RowLevelRangeLessThanFiterExecuterImpl.java | 122 ++--
 .../filter/executer/TrueFilterExecutor.java     |  17 +-
 .../executer/ValueBasedFilterExecuterImpl.java  |  16 +-
 .../scan/filter/intf/FilterOptimizerBasic.java  |  26 -
 .../filter/optimizer/RangeFilterOptmizer.java   |   3 +-
 .../resolver/ConditionalFilterResolverImpl.java |  18 +-
 .../resolver/RowLevelFilterResolverImpl.java    |   2 +-
 .../RowLevelRangeFilterResolverImpl.java        |   2 +-
 .../MeasureColumnResolvedFilterInfo.java        |  12 -
 .../TrueConditionalResolverImpl.java            |  16 +-
 .../visitor/RangeDictionaryColumnVisitor.java   |   2 -
 .../core/scan/model/CarbonQueryPlan.java        | 151 ----
 .../core/scan/model/ProjectionColumn.java       |  59 ++
 .../core/scan/model/ProjectionDimension.java    |  45 ++
 .../core/scan/model/ProjectionMeasure.java      |  46 ++
 .../carbondata/core/scan/model/QueryColumn.java |  66 --
 .../core/scan/model/QueryDimension.java         |  56 --
 .../core/scan/model/QueryMeasure.java           |  59 --
 .../carbondata/core/scan/model/QueryModel.java  | 120 +---
 .../core/scan/model/QueryProjection.java        |  83 +++
 .../processor/AbstractDataBlockIterator.java    | 251 -------
 .../core/scan/processor/BlockletIterator.java   |  14 +-
 .../core/scan/processor/BlocksChunkHolder.java  | 141 ----
 .../core/scan/processor/DataBlockIterator.java  | 269 +++++++
 .../scan/processor/RawBlockletColumnChunks.java | 113 +++
 .../processor/impl/DataBlockIteratorImpl.java   |  69 --
 .../core/scan/result/AbstractScannedResult.java | 698 -------------------
 .../core/scan/result/BatchResult.java           | 103 ---
 .../core/scan/result/BlockletScannedResult.java | 618 ++++++++++++++++
 .../carbondata/core/scan/result/RowBatch.java   | 103 +++
 .../result/impl/FilterQueryScannedResult.java   |  50 +-
 .../impl/NonFilterQueryScannedResult.java       |  14 +-
 .../AbstractDetailQueryResultIterator.java      |  44 +-
 .../scan/result/iterator/ChunkRowIterator.java  |   8 +-
 .../iterator/DetailQueryResultIterator.java     |  14 +-
 .../PartitionSpliterRawResultIterator.java      |  15 +-
 .../scan/result/iterator/RawResultIterator.java |  10 +-
 .../VectorDetailQueryResultIterator.java        |   4 +-
 .../scan/result/vector/CarbonColumnarBatch.java |   3 -
 .../scan/result/vector/ColumnVectorInfo.java    |   8 +-
 .../vector/MeasureDataVectorProcessor.java      |  36 +-
 .../scan/scanner/AbstractBlockletScanner.java   | 181 -----
 .../core/scan/scanner/BlockletScanner.java      |  22 +-
 .../scanner/impl/BlockletFilterScanner.java     | 329 +++++++++
 .../scan/scanner/impl/BlockletFullScanner.java  | 191 +++++
 .../core/scan/scanner/impl/FilterScanner.java   | 326 ---------
 .../scan/scanner/impl/NonFilterScanner.java     |  35 -
 .../core/scan/wrappers/ByteArrayWrapper.java    |  41 +-
 .../core/stats/PartitionStatistic.java          |  23 -
 .../core/stats/QueryStatisticsModel.java        |   3 -
 .../core/statusmanager/LoadMetadataDetails.java |   9 -
 .../statusmanager/SegmentStatusManager.java     |  16 -
 .../SegmentUpdateStatusManager.java             | 156 +----
 .../util/AbstractDataFileFooterConverter.java   |   1 -
 .../apache/carbondata/core/util/CarbonUtil.java |  34 +-
 .../core/util/DataFileFooterConverter.java      |   6 +-
 .../core/writer/CarbonDictionaryWriter.java     |  11 -
 .../core/writer/CarbonDictionaryWriterImpl.java |   2 +-
 .../DictionaryCacheLoaderImplTest.java          |   4 +-
 .../store/impl/DFSFileHolderImplUnitTest.java   | 149 ----
 .../store/impl/DFSFileReaderImplUnitTest.java   | 149 ++++
 .../store/impl/FileHolderImplUnitTest.java      | 154 ----
 .../store/impl/FileReaderImplUnitTest.java      | 154 ++++
 .../datastore/block/SegmentPropertiesTest.java  |   8 +-
 .../impl/ColumnGroupDimensionDataChunkTest.java |   8 +-
 .../impl/FixedLengthDimensionDataChunkTest.java |   8 +-
 .../impl/btree/BTreeBlockFinderTest.java        |  20 +-
 .../impl/RawBasedResultCollectorTest.java       |  32 +-
 .../core/scan/executor/util/QueryUtilTest.java  |  41 +-
 .../scan/executor/util/RestructureUtilTest.java |  31 +-
 .../core/scan/filter/FilterUtilTest.java        |   2 +-
 .../executer/ExcludeFilterExecuterImplTest.java |   6 +-
 .../executer/IncludeFilterExecuterImplTest.java |  32 +-
 .../core/scan/result/BatchResultTest.java       |  94 ---
 .../core/scan/result/RowBatchTest.java          |  94 +++
 .../carbondata/core/util/CarbonUtilTest.java    |  90 +--
 .../core/util/DataFileFooterConverterTest.java  |  11 +-
 .../core/util/RangeFilterProcessorTest.java     |   9 +-
 .../scanner/impl/FilterScannerTest.java         | 160 -----
 dev/findbugs-exclude.xml                        |   2 +-
 .../examples/CarbonSessionExample.scala         |   2 +-
 .../hadoop/api/CarbonTableInputFormat.java      |  44 +-
 .../internal/index/impl/InMemoryBTreeIndex.java |   4 +-
 .../streaming/CarbonStreamRecordReader.java     |   2 +-
 .../hadoop/util/BlockLevelTraverser.java        |   2 +-
 .../hadoop/util/CarbonInputFormatUtil.java      |  73 +-
 .../hive/MapredCarbonInputFormat.java           |   9 +-
 .../presto/CarbonVectorizedRecordReader.java    | 243 -------
 .../carbondata/presto/CarbondataPageSource.java |   2 +-
 .../presto/CarbondataRecordCursor.java          |   6 +-
 .../carbondata/presto/CarbondataRecordSet.java  |   4 +-
 .../presto/CarbondataRecordSetProvider.java     |   2 +-
 .../PrestoCarbonVectorizedRecordReader.java     | 243 +++++++
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   3 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   2 +-
 .../vectorreader/ColumnarVectorWrapper.java     |   2 +-
 .../VectorizedCarbonRecordReader.java           |  51 +-
 .../merger/CarbonCompactionExecutor.java        |  64 +-
 .../partition/impl/QueryPartitionHelper.java    |   6 +-
 .../spliter/AbstractCarbonQueryExecutor.java    |  54 +-
 .../partition/spliter/CarbonSplitExecutor.java  |   4 +-
 .../processing/util/CarbonQueryUtil.java        |   6 +-
 .../carbondata/streaming/StreamHandoffRDD.scala |   2 +-
 211 files changed, 5843 insertions(+), 7396 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
index 598d00e..fb67208 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
@@ -260,8 +260,7 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
     DictionaryCacheLoader dictionaryCacheLoader =
         new DictionaryCacheLoaderImpl(dictionaryColumnUniqueIdentifier);
     dictionaryCacheLoader
-        .load(dictionaryInfo, dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
-            dictionaryChunkStartOffset, dictionaryChunkEndOffset, loadSortIndex);
+        .load(dictionaryInfo, dictionaryChunkStartOffset, dictionaryChunkEndOffset, loadSortIndex);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
index 01c277d..8df1539 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
@@ -19,8 +19,6 @@ package org.apache.carbondata.core.cache.dictionary;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-
 public interface DictionaryCacheLoader {
 
   /**
@@ -28,7 +26,6 @@ public interface DictionaryCacheLoader {
    *
    * @param dictionaryInfo             dictionary info object which will hold the required data
    *                                   for a given column
-   * @param columnIdentifier           column unique identifier
    * @param dictionaryChunkStartOffset start offset from where dictionary file has to
    *                                   be read
    * @param dictionaryChunkEndOffset   end offset till where dictionary file has to
@@ -37,7 +34,7 @@ public interface DictionaryCacheLoader {
    *                                   read in memory after dictionary loading
    * @throws IOException
    */
-  void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
-      long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
+  void load(DictionaryInfo dictionaryInfo, long dictionaryChunkStartOffset,
+      long dictionaryChunkEndOffset, boolean loadSortIndex)
       throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
index a603c01..899abf5 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.reader.CarbonDictionaryReader;
 import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
 import org.apache.carbondata.core.service.CarbonCommonFactory;
@@ -43,8 +42,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
   /**
    * @param dictionaryColumnUniqueIdentifier dictionary column identifier
    */
-  public DictionaryCacheLoaderImpl(
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
+  DictionaryCacheLoaderImpl(DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     this.dictionaryColumnUniqueIdentifier = dictionaryColumnUniqueIdentifier;
   }
 
@@ -53,7 +51,6 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    *
    * @param dictionaryInfo             dictionary info object which will hold the required data
    *                                   for a given column
-   * @param columnIdentifier           column unique identifier
    * @param dictionaryChunkStartOffset start offset from where dictionary file has to
    *                                   be read
    * @param dictionaryChunkEndOffset   end offset till where dictionary file has to
@@ -62,9 +59,9 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    *                                   read in memory after dictionary loading
    * @throws IOException
    */
-  @Override public void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
-      long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
-      throws IOException {
+  @Override
+  public void load(DictionaryInfo dictionaryInfo, long dictionaryChunkStartOffset,
+      long dictionaryChunkEndOffset, boolean loadSortIndex) throws IOException {
     Iterator<byte[]> columnDictionaryChunkWrapper =
         load(dictionaryColumnUniqueIdentifier, dictionaryChunkStartOffset,
             dictionaryChunkEndOffset);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java
index 3bfbc45..381e764 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/BTreeBuilderInfo.java
@@ -43,16 +43,10 @@ public class BTreeBuilderInfo {
     this.footerList = footerList;
   }
 
-  /**
-   * @return the eachDimensionBlockSize
-   */
   public int[] getDimensionColumnValueSize() {
     return dimensionColumnValueSize;
   }
 
-  /**
-   * @return the footerList
-   */
   public List<DataFileFooter> getFooterList() {
     return footerList;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
index 13d5f69..273f833 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
@@ -18,7 +18,6 @@ package org.apache.carbondata.core.datastore;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 
@@ -28,46 +27,47 @@ import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 public interface DataRefNode {
 
   /**
-   * Method to get the next block this can be used while scanning when
+   * Return the next data block in the tree, this can be used while scanning when
    * iterator of this class can be used iterate over blocks
-   *
-   * @return next block
    */
   DataRefNode getNextDataRefNode();
 
   /**
-   * to get the number of keys tuples present in the block
-   *
-   * @return number of keys in the block
+   * Return the number of rows in the data block
    */
-  int nodeSize();
+  int numRows();
 
   /**
-   * Method can be used to get the block index .This can be used when multiple
-   * thread can be used scan group of blocks in that can we can assign the
+   * Return the block index. This can be used when multiple
+   * thread can be used scan group of blocks in that can we can assign
    * some of the blocks to one thread and some to other
-   *
-   * @return block number
    */
-  long nodeNumber();
+  long nodeIndex();
+
+  /**
+   * Return the blocklet index in the node
+   */
+  short blockletIndex();
 
   /**
-   * Method is used for retreiving the BlockletId.
-   * @return the blockletid related to the data block.
+   * Return the number of pages
    */
-  String blockletId();
+  int numberOfPages();
 
   /**
-   * This method will be used to get the max value of all the columns this can
+   * Return the number of rows for a give page
+   */
+  int getPageRowCount(int pageNumber);
+
+  /**
+   * Return the max value of all the columns, this can
    * be used in case of filter query
-   *
    */
   byte[][] getColumnsMaxValue();
 
   /**
-   * This method will be used to get the min value of all the columns this can
+   * Return the min value of all the columns, this can
    * be used in case of filter query
-   *
    */
   byte[][] getColumnsMinValue();
 
@@ -75,15 +75,15 @@ public interface DataRefNode {
    * Below method will be used to get the dimension chunks
    *
    * @param fileReader   file reader to read the chunks from file
-   * @param blockIndexes range indexes of the blocks need to be read
+   * @param columnIndexRange range indexes of the blocks need to be read
    *                     value can be {{0,10},{11,12},{13,13}}
    *                     here 0 to 10 and 11 to 12 column blocks will be read in one
    *                     IO operation 13th column block will be read separately
    *                     This will be helpful to reduce IO by reading bigger chunk of
-   *                     data in On IO
+   *                     data in one IO operation
    * @return dimension data chunks
    */
-  DimensionRawColumnChunk[] getDimensionChunks(FileHolder fileReader, int[][] blockIndexes)
+  DimensionRawColumnChunk[] readDimensionChunks(FileReader fileReader, int[][] columnIndexRange)
       throws IOException;
 
   /**
@@ -92,54 +92,31 @@ public interface DataRefNode {
    * @param fileReader file reader to read the chunk from file
    * @return dimension data chunk
    */
-  DimensionRawColumnChunk getDimensionChunk(FileHolder fileReader, int blockIndexes)
+  DimensionRawColumnChunk readDimensionChunk(FileReader fileReader, int columnIndex)
       throws IOException;
 
   /**
    * Below method will be used to get the measure chunk
    *
    * @param fileReader   file reader to read the chunk from file
-   * @param blockIndexes range indexes of the blocks need to be read
+   * @param columnIndexRange range indexes of the blocks need to be read
    *                     value can be {{0,10},{11,12},{13,13}}
    *                     here 0 to 10 and 11 to 12 column blocks will be read in one
    *                     IO operation 13th column block will be read separately
    *                     This will be helpful to reduce IO by reading bigger chunk of
-   *                     data in On IO
+   *                     data in one IO operation
    * @return measure column data chunk
    */
-  MeasureRawColumnChunk[] getMeasureChunks(FileHolder fileReader, int[][] blockIndexes)
+  MeasureRawColumnChunk[] readMeasureChunks(FileReader fileReader, int[][] columnIndexRange)
       throws IOException;
 
   /**
    * Below method will be used to read the measure chunk
    *
    * @param fileReader file read to read the file chunk
-   * @param blockIndex block index to be read from file
+   * @param columnIndex block index to be read from file
    * @return measure data chunk
    */
-  MeasureRawColumnChunk getMeasureChunk(FileHolder fileReader, int blockIndex) throws IOException;
+  MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) throws IOException;
 
-  /**
-   * @param deleteDeltaDataCache
-   */
-  void setDeleteDeltaDataCache(BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache);
-
-  /**
-   * @return
-   */
-  BlockletLevelDeleteDeltaDataCache getDeleteDeltaDataCache();
-
-  /**
-   * number of pages in blocklet
-   * @return
-   */
-  int numberOfPages();
-
-  /**
-   * Return the number of rows for a give page
-   *
-   * @param pageNumber
-   * @return
-   */
-  int getPageRowCount(int pageNumber);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java b/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
deleted file mode 100644
index b4130a0..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/FileHolder.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.datastore;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-public interface FileHolder {
-
-  /**
-   * This method will be used to reads the data to byteBuffer from file based on offset
-   * and length(number of bytes) need to read
-   *
-   * @param filePath fully qualified file path
-   * @param offset reading start position,
-   * @param length number of bytes to be read
-   * @return ByteBuffer
-   * @throws IOException
-   */
-  ByteBuffer readByteBuffer(String filePath, long offset, int length)
-      throws IOException;
-  /**
-   * This method will be used to read the byte array from file based on offset
-   * and length(number of bytes) need to read
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @param length   number of bytes to be read
-   * @return read byte array
-   */
-  byte[] readByteArray(String filePath, long offset, int length) throws IOException;
-
-  /**
-   * This method will be used to read the byte array from file based on length(number of bytes)
-   *
-   * @param filePath fully qualified file path
-   * @param length   number of bytes to be read
-   * @return read byte array
-   */
-  byte[] readByteArray(String filePath, int length) throws IOException;
-
-  /**
-   * This method will be used to read int from file from postion(offset), here
-   * length will be always 4 bacause int byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read int
-   */
-  int readInt(String filePath, long offset) throws IOException;
-
-  /**
-   * This method will be used to read long from file from postion(offset), here
-   * length will be always 8 bacause int byte size is 8
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read long
-   */
-  long readLong(String filePath, long offset) throws IOException;
-
-  /**
-   * This method will be used to read int from file from postion(offset), here
-   * length will be always 4 bacause int byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @return read int
-   */
-  int readInt(String filePath) throws IOException;
-
-  /**
-   * This method will be used to read long value from file from postion(offset), here
-   * length will be always 8 because long byte size if 4
-   *
-   * @param filePath fully qualified file path
-   * @param offset   reading start position,
-   * @return read long
-   */
-  long readDouble(String filePath, long offset) throws IOException;
-
-  /**
-   * This method will be used to close all the streams currently present in the cache
-   */
-  void finish() throws IOException;
-
-  void setQueryId(String queryId);
-
-  String getQueryId();
-
-  /**
-   * Set the flag to read data page by page instead of whole blocklet.
-   *
-   * @param isReadPageByPage
-   */
-  void setReadPageByPage(boolean isReadPageByPage);
-
-  /**
-   * Whether read the data page by page from carbondata file instead of reading whole
-   * blocklet to memory. It is need in case of memory constraint operations.
-   */
-  boolean isReadPageByPage();
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java
new file mode 100644
index 0000000..df0d745
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/FileReader.java
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.datastore;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+public interface FileReader {
+
+  /**
+   * This method will be used to reads the data to byteBuffer from file based on offset
+   * and length(number of bytes) need to read
+   *
+   * @param filePath fully qualified file path
+   * @param offset reading start position,
+   * @param length number of bytes to be read
+   * @return ByteBuffer
+   * @throws IOException
+   */
+  ByteBuffer readByteBuffer(String filePath, long offset, int length)
+      throws IOException;
+  /**
+   * This method will be used to read the byte array from file based on offset
+   * and length(number of bytes) need to read
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @param length   number of bytes to be read
+   * @return read byte array
+   */
+  byte[] readByteArray(String filePath, long offset, int length) throws IOException;
+
+  /**
+   * This method will be used to read the byte array from file based on length(number of bytes)
+   *
+   * @param filePath fully qualified file path
+   * @param length   number of bytes to be read
+   * @return read byte array
+   */
+  byte[] readByteArray(String filePath, int length) throws IOException;
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read int
+   */
+  int readInt(String filePath, long offset) throws IOException;
+
+  /**
+   * This method will be used to read long from file from postion(offset), here
+   * length will be always 8 bacause int byte size is 8
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read long
+   */
+  long readLong(String filePath, long offset) throws IOException;
+
+  /**
+   * This method will be used to read int from file from postion(offset), here
+   * length will be always 4 bacause int byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @return read int
+   */
+  int readInt(String filePath) throws IOException;
+
+  /**
+   * This method will be used to read long value from file from postion(offset), here
+   * length will be always 8 because long byte size if 4
+   *
+   * @param filePath fully qualified file path
+   * @param offset   reading start position,
+   * @return read long
+   */
+  long readDouble(String filePath, long offset) throws IOException;
+
+  /**
+   * This method will be used to close all the streams currently present in the cache
+   */
+  void finish() throws IOException;
+
+  /**
+   * Set the flag to read data page by page instead of whole blocklet.
+   *
+   * @param isReadPageByPage
+   */
+  void setReadPageByPage(boolean isReadPageByPage);
+
+  /**
+   * Whether read the data page by page from carbondata file instead of reading whole
+   * blocklet to memory. It is need in case of memory constraint operations.
+   */
+  boolean isReadPageByPage();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
index 5f82e87..c134db9 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/SegmentProperties.java
@@ -94,10 +94,9 @@ public class SegmentProperties {
   private int[] complexDimColumnCardinality;
 
   /**
-   * mapping of dimension column to block in a file this will be used for
-   * reading the blocks from file
+   * mapping of dimension ordinal in schema to column chunk index in the data file
    */
-  private Map<Integer, Integer> dimensionOrdinalToBlockMapping;
+  private Map<Integer, Integer> dimensionOrdinalToChunkMapping;
 
   /**
    * a block can have multiple columns. This will have block index as key
@@ -106,10 +105,9 @@ public class SegmentProperties {
   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
+   * mapping of measure ordinal in schema to column chunk index in the data file
    */
-  private Map<Integer, Integer> measuresOrdinalToBlockMapping;
+  private Map<Integer, Integer> measuresOrdinalToChunkMapping;
 
   /**
    * size of the each dimension column value in a block this can be used when
@@ -172,15 +170,15 @@ public class SegmentProperties {
         new ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     measures = new ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     fillDimensionAndMeasureDetails(columnsInTable, columnCardinality);
-    dimensionOrdinalToBlockMapping =
+    dimensionOrdinalToChunkMapping =
         new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     blockTodimensionOrdinalMapping =
         new HashMap<Integer, Set<Integer>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    measuresOrdinalToBlockMapping =
+    measuresOrdinalToChunkMapping =
         new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     intialiseColGroups();
     fillOrdinalToBlockMappingForDimension();
-    fillOrdinalToBlockIndexMappingForMeasureColumns();
+    fillOrdinalToChunkIndexMappingForMeasureColumns();
     fillColumnGroupAndItsCardinality(columnCardinality);
     fillKeyGeneratorDetails();
   }
@@ -237,7 +235,7 @@ public class SegmentProperties {
       if (dimension.isColumnar() || dimension.columnGroupId() != prvcolumnGroupId) {
         blockOrdinal++;
       }
-      dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), blockOrdinal);
+      dimensionOrdinalToChunkMapping.put(dimension.getOrdinal(), blockOrdinal);
       prvcolumnGroupId = dimension.columnGroupId();
       index++;
     }
@@ -245,7 +243,7 @@ public class SegmentProperties {
     // complex dimension will be stored at last
     while (index < complexDimensions.size()) {
       dimension = complexDimensions.get(index);
-      dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), ++blockOrdinal);
+      dimensionOrdinalToChunkMapping.put(dimension.getOrdinal(), ++blockOrdinal);
       blockOrdinal = fillComplexDimensionChildBlockIndex(blockOrdinal, dimension);
       index++;
     }
@@ -256,7 +254,7 @@ public class SegmentProperties {
    *
    */
   private void fillBlockToDimensionOrdinalMapping() {
-    Set<Entry<Integer, Integer>> blocks = dimensionOrdinalToBlockMapping.entrySet();
+    Set<Entry<Integer, Integer>> blocks = dimensionOrdinalToChunkMapping.entrySet();
     Iterator<Entry<Integer, Integer>> blockItr = blocks.iterator();
     while (blockItr.hasNext()) {
       Entry<Integer, Integer> block = blockItr.next();
@@ -280,7 +278,7 @@ public class SegmentProperties {
    */
   private int fillComplexDimensionChildBlockIndex(int blockOrdinal, CarbonDimension dimension) {
     for (int i = 0; i < dimension.getNumberOfChild(); i++) {
-      dimensionOrdinalToBlockMapping
+      dimensionOrdinalToChunkMapping
           .put(dimension.getListOfChildDimensions().get(i).getOrdinal(), ++blockOrdinal);
       if (dimension.getListOfChildDimensions().get(i).getNumberOfChild() > 0) {
         blockOrdinal = fillComplexDimensionChildBlockIndex(blockOrdinal,
@@ -295,11 +293,11 @@ public class SegmentProperties {
    * of measure ordinal to its block index mapping in
    * file
    */
-  private void fillOrdinalToBlockIndexMappingForMeasureColumns() {
+  private void fillOrdinalToChunkIndexMappingForMeasureColumns() {
     int blockOrdinal = 0;
     int index = 0;
     while (index < measures.size()) {
-      measuresOrdinalToBlockMapping.put(measures.get(index).getOrdinal(), blockOrdinal);
+      measuresOrdinalToChunkMapping.put(measures.get(index).getOrdinal(), blockOrdinal);
       blockOrdinal++;
       index++;
     }
@@ -731,17 +729,17 @@ public class SegmentProperties {
   }
 
   /**
-   * @return the dimensionOrdinalToBlockMapping
+   * @return the dimensionOrdinalToChunkMapping
    */
-  public Map<Integer, Integer> getDimensionOrdinalToBlockMapping() {
-    return dimensionOrdinalToBlockMapping;
+  public Map<Integer, Integer> getDimensionOrdinalToChunkMapping() {
+    return dimensionOrdinalToChunkMapping;
   }
 
   /**
-   * @return the measuresOrdinalToBlockMapping
+   * @return the measuresOrdinalToChunkMapping
    */
-  public Map<Integer, Integer> getMeasuresOrdinalToBlockMapping() {
-    return measuresOrdinalToBlockMapping;
+  public Map<Integer, Integer> getMeasuresOrdinalToChunkMapping() {
+    return measuresOrdinalToChunkMapping;
   }
 
   /**
@@ -805,16 +803,6 @@ public class SegmentProperties {
   }
 
   /**
-   * 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
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java
deleted file mode 100644
index 3791314..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnDataChunk.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datastore.chunk;
-
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-
-/**
- * Interface for dimension column chunk.
- */
-public interface DimensionColumnDataChunk {
-
-  /**
-   * Below method will be used to fill the data based on offset and row id
-   *
-   * @param data   data to filed
-   * @param offset offset from which data need to be filed
-   * @return how many bytes was copied
-   */
-  int fillChunkData(byte[] data, int offset, int columnIndex, KeyStructureInfo restructuringInfo);
-
-  /**
-   * It uses to convert column data to dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo @return
-   */
-  int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Fill the data to vector
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Fill the data to vector
-   * @param rowMapping
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo);
-
-  /**
-   * Below method to get  the data based in row id
-   *
-   * @return chunk
-   */
-  byte[] getChunkData(int columnIndex);
-
-  /**
-   * @return inverted index
-   */
-  int getInvertedIndex(int index);
-
-  /**
-   *
-   * @param invertedIndex
-   * @return index reverse index
-   */
-  int getInvertedReverseIndex(int invertedIndex);
-
-  /**
-   * @return whether column is dictionary column or not
-   */
-  boolean isNoDicitionaryColumn();
-
-  /**
-   * @return length of each column
-   */
-  int getColumnValueSize();
-
-  /**
-   * @return whether columns where explictly sorted or not
-   */
-  boolean isExplicitSorted();
-
-  /**
-   * to compare the data
-   *
-   * @param index        row index to be compared
-   * @param compareValue value to compare
-   * @return compare result
-   */
-  int compareTo(int index, byte[] compareValue);
-
-  /**
-   * below method will be used to free the allocated memory
-   */
-  void freeMemory();
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java
new file mode 100644
index 0000000..15840bc
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/DimensionColumnPage.java
@@ -0,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.apache.carbondata.core.datastore.chunk;
+
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+
+/**
+ * Interface for dimension column chunk.
+ */
+public interface DimensionColumnPage {
+
+  /**
+   * Below method will be used to fill the data based on offset and row id
+   *
+   * @param offset offset from which data need to be filed
+   * @param data   data to filed
+   * @return how many bytes was copied
+   */
+  int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo restructuringInfo);
+
+  /**
+   * It uses to convert column data to dictionary integer value
+   *
+   * @param rowId
+   * @param chunkIndex
+   * @param outputSurrogateKey
+   * @param restructuringInfo @return
+   */
+  int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
+      KeyStructureInfo restructuringInfo);
+
+  /**
+   * Fill the data to vector
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo);
+
+  /**
+   * Fill the data to vector
+   * @param filteredRowId
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo);
+
+  /**
+   * Below method to get  the data based in row id
+   *
+   * @return chunk
+   */
+  byte[] getChunkData(int rowId);
+
+  /**
+   * @return inverted index
+   */
+  int getInvertedIndex(int rowId);
+
+  /**
+   *
+   * @param rowId
+   * @return index reverse index
+   */
+  int getInvertedReverseIndex(int rowId);
+
+  /**
+   * @return whether column is dictionary column or not
+   */
+  boolean isNoDicitionaryColumn();
+
+  /**
+   * @return whether columns where explictly sorted or not
+   */
+  boolean isExplicitSorted();
+
+  /**
+   * to compare the data
+   *
+   * @param rowId        row index to be compared
+   * @param compareValue value to compare
+   * @return compare result
+   */
+  int compareTo(int rowId, byte[] compareValue);
+
+  /**
+   * below method will be used to free the allocated memory
+   */
+  void freeMemory();
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java
new file mode 100644
index 0000000..6f316c5
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionColumnPage.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datastore.chunk.impl;
+
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionDataChunkStore;
+
+/**
+ * Class responsibility is to give access to dimension column data chunk store
+ */
+public abstract class AbstractDimensionColumnPage implements DimensionColumnPage {
+
+  /**
+   * data chunks
+   */
+  DimensionDataChunkStore dataChunkStore;
+
+  /**
+   * @return whether columns where explicitly sorted or not
+   */
+  @Override public boolean isExplicitSorted() {
+    return dataChunkStore.isExplicitSorted();
+  }
+
+  /**
+   * Below method to get the data based in row id
+   *
+   * @param rowId row id of the data
+   * @return chunk
+   */
+  @Override public byte[] getChunkData(int rowId) {
+    return dataChunkStore.getRow(rowId);
+  }
+
+  /**
+   * @return inverted index
+   */
+  @Override public int getInvertedIndex(int rowId) {
+    return dataChunkStore.getInvertedIndex(rowId);
+  }
+
+  /**
+   * @param rowId
+   * @return inverted index reverse
+   */
+  @Override public int getInvertedReverseIndex(int rowId) {
+    return dataChunkStore.getInvertedReverseIndex(rowId);
+  }
+
+  /**
+   * To compare the data
+   *
+   * @param rowId        row index to be compared
+   * @param compareValue value to compare
+   * @return compare result
+   */
+  @Override public int compareTo(int rowId, byte[] compareValue) {
+    // TODO Auto-generated method stub
+    return dataChunkStore.compareTo(rowId, compareValue);
+  }
+
+  /**
+   * below method will be used to free the allocated memory
+   */
+  @Override public void freeMemory() {
+    dataChunkStore.freeMemory();
+  }
+
+  /**
+   * @return column is dictionary column or not
+   */
+  @Override public boolean isNoDicitionaryColumn() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java
deleted file mode 100644
index eac062f..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/AbstractDimensionDataChunk.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datastore.chunk.impl;
-
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.store.DimensionDataChunkStore;
-
-/**
- * Class responsibility is to give access to dimension column data chunk store
- */
-public abstract class AbstractDimensionDataChunk implements DimensionColumnDataChunk {
-
-  /**
-   * data chunks
-   */
-  protected DimensionDataChunkStore dataChunkStore;
-
-  /**
-   * @return whether columns where explicitly sorted or not
-   */
-  @Override public boolean isExplicitSorted() {
-    return dataChunkStore.isExplicitSorted();
-  }
-
-  /**
-   * Below method to get the data based in row id
-   *
-   * @param index row id of the data
-   * @return chunk
-   */
-  @Override public byte[] getChunkData(int index) {
-    return dataChunkStore.getRow(index);
-  }
-
-  /**
-   * @return inverted index
-   */
-  @Override public int getInvertedIndex(int index) {
-    return dataChunkStore.getInvertedIndex(index);
-  }
-
-  /**
-   * @param invertedIndex
-   * @return inverted index reverse
-   */
-  @Override public int getInvertedReverseIndex(int invertedIndex) {
-    return dataChunkStore.getInvertedReverseIndex(invertedIndex);
-  }
-  /**
-   * @return length of each column
-   */
-  @Override public int getColumnValueSize() {
-    return dataChunkStore.getColumnValueSize();
-  }
-
-  /**
-   * To compare the data
-   *
-   * @param index        row index to be compared
-   * @param compareValue value to compare
-   * @return compare result
-   */
-  @Override public int compareTo(int index, byte[] compareValue) {
-    // TODO Auto-generated method stub
-    return dataChunkStore.compareTo(index, compareValue);
-  }
-
-  /**
-   * below method will be used to free the allocated memory
-   */
-  @Override public void freeMemory() {
-    dataChunkStore.freeMemory();
-  }
-
-  /**
-   * @return column is dictionary column or not
-   */
-  @Override public boolean isNoDicitionaryColumn() {
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java
new file mode 100644
index 0000000..741c13d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionColumnPage.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datastore.chunk.impl;
+
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+
+/**
+ * This class is gives access to column group dimension data chunk store
+ */
+public class ColumnGroupDimensionColumnPage extends AbstractDimensionColumnPage {
+
+  /**
+   * Constructor for this class
+   *
+   * @param dataChunk       data chunk
+   * @param columnValueSize chunk attributes
+   * @param numberOfRows
+   */
+  public ColumnGroupDimensionColumnPage(byte[] dataChunk, int columnValueSize, int numberOfRows) {
+    this.dataChunkStore = DimensionChunkStoreFactory.INSTANCE
+        .getDimensionChunkStore(columnValueSize, false, numberOfRows, dataChunk.length,
+        DimensionStoreType.FIXEDLENGTH);
+    this.dataChunkStore.putArray(null, null, dataChunk);
+  }
+
+  /**
+   * Below method will be used to fill the data based on offset and row id
+   *
+   * @param rowId             row id of the chunk
+   * @param offset            offset from which data need to be filed
+   * @param data              data to filed
+   * @param restructuringInfo define the structure of the key
+   * @return how many bytes was copied
+   */
+  @Override public int fillRawData(int rowId, int offset, byte[] data,
+      KeyStructureInfo restructuringInfo) {
+    byte[] row = dataChunkStore.getRow(rowId);
+    byte[] maskedKey = getMaskedKey(row, restructuringInfo);
+    System.arraycopy(maskedKey, 0, data, offset, maskedKey.length);
+    return maskedKey.length;
+  }
+
+  /**
+   * Converts to column dictionary integer value
+   *
+   * @param rowId
+   * @param chunkIndex
+   * @param outputSurrogateKey
+   * @param info          KeyStructureInfo
+   * @return
+   */
+  @Override public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
+      KeyStructureInfo info) {
+    byte[] data = dataChunkStore.getRow(rowId);
+    long[] keyArray = info.getKeyGenerator().getKeyArray(data);
+    int[] ordinal = info.getMdkeyQueryDimensionOrdinal();
+    for (int i = 0; i < ordinal.length; i++) {
+      outputSurrogateKey[chunkIndex++] = (int) keyArray[ordinal[i]];
+    }
+    return chunkIndex;
+  }
+
+  /**
+   * Below method will be used to get the masked key
+   *
+   * @param data   data
+   * @param info
+   * @return
+   */
+  private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) {
+    byte[] maskedKey = new byte[info.getMaskByteRanges().length];
+    int counter = 0;
+    int byteRange = 0;
+    for (int i = 0; i < info.getMaskByteRanges().length; i++) {
+      byteRange = info.getMaskByteRanges()[i];
+      maskedKey[counter++] = (byte) (data[byteRange] & info.getMaxKey()[byteRange]);
+    }
+    return maskedKey;
+  }
+
+  /**
+   * @return inverted index
+   */
+  @Override public int getInvertedIndex(int rowId) {
+    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
+  }
+
+  /**
+   * @param rowId
+   * @return inverted index reverse
+   */
+  @Override public int getInvertedReverseIndex(int rowId) {
+    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
+  }
+
+  /**
+   * @return whether columns where explictly sorted or not
+   */
+  @Override public boolean isExplicitSorted() {
+    return false;
+  }
+
+  /**
+   * to compare the data
+   *
+   * @param rowId        row index to be compared
+   * @param compareValue value to compare
+   * @return compare result
+   */
+  @Override public int compareTo(int rowId, byte[] compareValue) {
+    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = offset + columnVectorInfo.size;
+    int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal();
+    for (int k = offset; k < len; k++) {
+      long[] keyArray = restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(k));
+      int index = 0;
+      for (int i = chunkIndex; i < chunkIndex + ordinal.length; i++) {
+        if (vectorInfo[i].directDictionaryGenerator == null) {
+          vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]);
+        } else {
+          vectorInfo[i].vector.putLong(vectorOffset, (long) vectorInfo[i].directDictionaryGenerator
+              .getValueFromSurrogate((int) keyArray[ordinal[index++]]));
+        }
+      }
+      vectorOffset++;
+    }
+    return chunkIndex + ordinal.length;
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param filteredRowId
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo,
+      int chunkIndex, KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = offset + columnVectorInfo.size;
+    int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal();
+    for (int k = offset; k < len; k++) {
+      long[] keyArray =
+          restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(filteredRowId[k]));
+      int index = 0;
+      for (int i = chunkIndex; i < chunkIndex + ordinal.length; i++) {
+        if (vectorInfo[i].directDictionaryGenerator == null) {
+          vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]);
+        } else {
+          vectorInfo[i].vector.putLong(vectorOffset, (long) vectorInfo[i].directDictionaryGenerator
+              .getValueFromSurrogate((int) keyArray[ordinal[index++]]));
+        }
+      }
+      vectorOffset++;
+    }
+    return chunkIndex + ordinal.length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
deleted file mode 100644
index b76ae53..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datastore.chunk.impl;
-
-import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
-import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-
-/**
- * This class is gives access to column group dimension data chunk store
- */
-public class ColumnGroupDimensionDataChunk extends AbstractDimensionDataChunk {
-
-  /**
-   * Constructor for this class
-   *
-   * @param dataChunk       data chunk
-   * @param columnValueSize chunk attributes
-   * @param numberOfRows
-   */
-  public ColumnGroupDimensionDataChunk(byte[] dataChunk, int columnValueSize, int numberOfRows) {
-    this.dataChunkStore = DimensionChunkStoreFactory.INSTANCE
-        .getDimensionChunkStore(columnValueSize, false, numberOfRows, dataChunk.length,
-        DimensionStoreType.FIXEDLENGTH);
-    this.dataChunkStore.putArray(null, null, dataChunk);
-  }
-
-  /**
-   * Below method will be used to fill the data based on offset and row id
-   *
-   * @param data              data to filed
-   * @param offset            offset from which data need to be filed
-   * @param rowId             row id of the chunk
-   * @param restructuringInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int rowId,
-      KeyStructureInfo restructuringInfo) {
-    byte[] row = dataChunkStore.getRow(rowId);
-    byte[] maskedKey = getMaskedKey(row, restructuringInfo);
-    System.arraycopy(maskedKey, 0, data, offset, maskedKey.length);
-    return maskedKey.length;
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param info          KeyStructureInfo
-   * @return
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo info) {
-    byte[] data = dataChunkStore.getRow(rowId);
-    long[] keyArray = info.getKeyGenerator().getKeyArray(data);
-    int[] ordinal = info.getMdkeyQueryDimensionOrdinal();
-    for (int i = 0; i < ordinal.length; i++) {
-      row[columnIndex++] = (int) keyArray[ordinal[i]];
-    }
-    return columnIndex;
-  }
-
-  /**
-   * Below method will be used to get the masked key
-   *
-   * @param data   data
-   * @param info
-   * @return
-   */
-  private byte[] getMaskedKey(byte[] data, KeyStructureInfo info) {
-    byte[] maskedKey = new byte[info.getMaskByteRanges().length];
-    int counter = 0;
-    int byteRange = 0;
-    for (int i = 0; i < info.getMaskByteRanges().length; i++) {
-      byteRange = info.getMaskByteRanges()[i];
-      maskedKey[counter++] = (byte) (data[byteRange] & info.getMaxKey()[byteRange]);
-    }
-    return maskedKey;
-  }
-
-  /**
-   * @return inverted index
-   */
-  @Override public int getInvertedIndex(int index) {
-    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
-  }
-
-  /**
-   * @param invertedIndex
-   * @return inverted index reverse
-   */
-  @Override public int getInvertedReverseIndex(int invertedIndex) {
-    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
-  }
-
-  /**
-   * @return whether columns where explictly sorted or not
-   */
-  @Override public boolean isExplicitSorted() {
-    return false;
-  }
-
-  /**
-   * to compare the data
-   *
-   * @param index        row index to be compared
-   * @param compareValue value to compare
-   * @return compare result
-   */
-  @Override public int compareTo(int index, byte[] compareValue) {
-    throw new UnsupportedOperationException("Operation not supported in case of cloumn group");
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo) {
-    ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = offset + columnVectorInfo.size;
-    int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal();
-    for (int k = offset; k < len; k++) {
-      long[] keyArray = restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(k));
-      int index = 0;
-      for (int i = column; i < column + ordinal.length; i++) {
-        if (vectorInfo[i].directDictionaryGenerator == null) {
-          vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]);
-        } else {
-          vectorInfo[i].vector.putLong(vectorOffset, (long) vectorInfo[i].directDictionaryGenerator
-              .getValueFromSurrogate((int) keyArray[ordinal[index++]]));
-        }
-      }
-      vectorOffset++;
-    }
-    return column + ordinal.length;
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param rowMapping
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo,
-      int column, KeyStructureInfo restructuringInfo) {
-    ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = offset + columnVectorInfo.size;
-    int[] ordinal = restructuringInfo.getMdkeyQueryDimensionOrdinal();
-    for (int k = offset; k < len; k++) {
-      long[] keyArray =
-          restructuringInfo.getKeyGenerator().getKeyArray(dataChunkStore.getRow(rowMapping[k]));
-      int index = 0;
-      for (int i = column; i < column + ordinal.length; i++) {
-        if (vectorInfo[i].directDictionaryGenerator == null) {
-          vectorInfo[i].vector.putInt(vectorOffset, (int) keyArray[ordinal[index++]]);
-        } else {
-          vectorInfo[i].vector.putLong(vectorOffset, (long) vectorInfo[i].directDictionaryGenerator
-              .getValueFromSurrogate((int) keyArray[ordinal[index++]]));
-        }
-      }
-      vectorOffset++;
-    }
-    return column + ordinal.length;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java
index 705c13c..f9bb590 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/DimensionRawColumnChunk.java
@@ -19,25 +19,25 @@ package org.apache.carbondata.core.datastore.chunk.impl;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.AbstractRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
 import org.apache.carbondata.core.memory.MemoryException;
 
 /**
  * Contains raw dimension data,
  * 1. The read uncompressed raw data of column chunk with all pages is stored in this instance.
- * 2. The raw data can be converted to processed chunk using convertToDimColDataChunk method
+ * 2. The raw data can be converted to processed chunk using decodeColumnPage method
  *  by specifying page number.
  */
 public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
 
-  private DimensionColumnDataChunk[] dataChunks;
+  private DimensionColumnPage[] dataChunks;
 
   private DimensionColumnChunkReader chunkReader;
 
-  private FileHolder fileHolder;
+  private FileReader fileReader;
 
   public DimensionRawColumnChunk(int columnIndex, ByteBuffer rawData, long offSet, int length,
       DimensionColumnChunkReader columnChunkReader) {
@@ -46,17 +46,17 @@ public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
   }
 
   /**
-   * Convert all raw data with all pages to processed DimensionColumnDataChunk's
+   * Convert all raw data with all pages to processed DimensionColumnPage's
    * @return
    */
-  public DimensionColumnDataChunk[] convertToDimColDataChunks() {
+  public DimensionColumnPage[] decodeAllColumnPages() {
     if (dataChunks == null) {
-      dataChunks = new DimensionColumnDataChunk[pagesCount];
+      dataChunks = new DimensionColumnPage[pagesCount];
     }
     for (int i = 0; i < pagesCount; i++) {
       try {
         if (dataChunks[i] == null) {
-          dataChunks[i] = chunkReader.convertToDimensionChunk(this, i);
+          dataChunks[i] = chunkReader.decodeColumnPage(this, i);
         }
       } catch (IOException | MemoryException e) {
         throw new RuntimeException(e);
@@ -66,24 +66,24 @@ public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
   }
 
   /**
-   * Convert raw data with specified page number processed to DimensionColumnDataChunk
-   * @param index
+   * Convert raw data with specified page number processed to DimensionColumnPage
+   * @param pageNumber
    * @return
    */
-  public DimensionColumnDataChunk convertToDimColDataChunk(int index) {
-    assert index < pagesCount;
+  public DimensionColumnPage decodeColumnPage(int pageNumber) {
+    assert pageNumber < pagesCount;
     if (dataChunks == null) {
-      dataChunks = new DimensionColumnDataChunk[pagesCount];
+      dataChunks = new DimensionColumnPage[pagesCount];
     }
-    if (dataChunks[index] == null) {
+    if (dataChunks[pageNumber] == null) {
       try {
-        dataChunks[index] = chunkReader.convertToDimensionChunk(this, index);
+        dataChunks[pageNumber] = chunkReader.decodeColumnPage(this, pageNumber);
       } catch (IOException | MemoryException e) {
         throw new RuntimeException(e);
       }
     }
 
-    return dataChunks[index];
+    return dataChunks[pageNumber];
   }
 
   /**
@@ -92,10 +92,10 @@ public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
    * @param index
    * @return
    */
-  public DimensionColumnDataChunk convertToDimColDataChunkWithOutCache(int index) {
+  public DimensionColumnPage convertToDimColDataChunkWithOutCache(int index) {
     assert index < pagesCount;
     try {
-      return chunkReader.convertToDimensionChunk(this, index);
+      return chunkReader.decodeColumnPage(this, index);
     } catch (Exception e) {
       throw new RuntimeException(e);
     }
@@ -111,11 +111,11 @@ public class DimensionRawColumnChunk extends AbstractRawColumnChunk {
     }
   }
 
-  public void setFileHolder(FileHolder fileHolder) {
-    this.fileHolder = fileHolder;
+  public void setFileReader(FileReader fileReader) {
+    this.fileReader = fileReader;
   }
 
-  public FileHolder getFileReader() {
-    return fileHolder;
+  public FileReader getFileReader() {
+    return fileReader;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java
new file mode 100644
index 0000000..ff54b12
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionColumnPage.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datastore.chunk.impl;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+
+/**
+ * This class is gives access to fixed length dimension data chunk store
+ */
+public class FixedLengthDimensionColumnPage extends AbstractDimensionColumnPage {
+
+  /**
+   * Constructor
+   *
+   * @param dataChunk            data chunk
+   * @param invertedIndex        inverted index
+   * @param invertedIndexReverse reverse inverted index
+   * @param numberOfRows         number of rows
+   * @param columnValueSize      size of each column value
+   */
+  public FixedLengthDimensionColumnPage(byte[] dataChunk, int[] invertedIndex,
+      int[] invertedIndexReverse, int numberOfRows, int columnValueSize) {
+    long totalSize = null != invertedIndex ?
+        dataChunk.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) :
+        dataChunk.length;
+    dataChunkStore = DimensionChunkStoreFactory.INSTANCE
+        .getDimensionChunkStore(columnValueSize, null != invertedIndex, numberOfRows, totalSize,
+            DimensionStoreType.FIXEDLENGTH);
+    dataChunkStore.putArray(invertedIndex, invertedIndexReverse, dataChunk);
+  }
+
+  /**
+   * Below method will be used to fill the data based on offset and row id
+   *
+   * @param rowId            row id of the chunk
+   * @param offset           offset from which data need to be filed
+   * @param data             data to filed
+   * @param keyStructureInfo define the structure of the key
+   * @return how many bytes was copied
+   */
+  @Override public int fillRawData(int rowId, int offset, byte[] data,
+      KeyStructureInfo keyStructureInfo) {
+    dataChunkStore.fillRow(rowId, data, offset);
+    return dataChunkStore.getColumnValueSize();
+  }
+
+  /**
+   * Converts to column dictionary integer value
+   *
+   * @param rowId
+   * @param chunkIndex
+   * @param outputSurrogateKey
+   * @param restructuringInfo
+   * @return
+   */
+  @Override public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
+      KeyStructureInfo restructuringInfo) {
+    outputSurrogateKey[chunkIndex] = dataChunkStore.getSurrogate(rowId);
+    return chunkIndex + 1;
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = columnVectorInfo.size + offset;
+    CarbonColumnVector vector = columnVectorInfo.vector;
+    for (int j = offset; j < len; j++) {
+      int dict = dataChunkStore.getSurrogate(j);
+      if (columnVectorInfo.directDictionaryGenerator == null) {
+        vector.putInt(vectorOffset++, dict);
+      } else {
+        Object valueFromSurrogate =
+            columnVectorInfo.directDictionaryGenerator.getValueFromSurrogate(dict);
+        if (valueFromSurrogate == null) {
+          vector.putNull(vectorOffset++);
+        } else {
+          DataType dataType = columnVectorInfo.directDictionaryGenerator.getReturnType();
+          if (dataType == DataTypes.INT) {
+            vector.putInt(vectorOffset++, (int) valueFromSurrogate);
+          } else if (dataType == DataTypes.LONG) {
+            vector.putLong(vectorOffset++, (long) valueFromSurrogate);
+          } else {
+            throw new IllegalArgumentException("unsupported data type: " +
+                columnVectorInfo.directDictionaryGenerator.getReturnType());
+          }
+        }
+      }
+    }
+    return chunkIndex + 1;
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param filteredRowId
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo,
+      int chunkIndex, KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = columnVectorInfo.size + offset;
+    CarbonColumnVector vector = columnVectorInfo.vector;
+    for (int j = offset; j < len; j++) {
+      int dict = dataChunkStore.getSurrogate(filteredRowId[j]);
+      if (columnVectorInfo.directDictionaryGenerator == null) {
+        vector.putInt(vectorOffset++, dict);
+      } else {
+        Object valueFromSurrogate =
+            columnVectorInfo.directDictionaryGenerator.getValueFromSurrogate(dict);
+        if (valueFromSurrogate == null) {
+          vector.putNull(vectorOffset++);
+        } else {
+          DataType dataType = columnVectorInfo.directDictionaryGenerator.getReturnType();
+          if (dataType == DataTypes.INT) {
+            vector.putInt(vectorOffset++, (int) valueFromSurrogate);
+          } else if (dataType == DataTypes.LONG) {
+            vector.putLong(vectorOffset++, (long) valueFromSurrogate);
+          } else {
+            throw new IllegalArgumentException("unsupported data type: " +
+                columnVectorInfo.directDictionaryGenerator.getReturnType());
+          }
+        }
+      }
+    }
+    return chunkIndex + 1;
+  }
+}


[26/49] carbondata git commit: [CARBONDATA-1544][Datamap] Datamap FineGrain implementation

Posted by ja...@apache.org.
[CARBONDATA-1544][Datamap] Datamap FineGrain implementation

Implemented interfaces for FG datamap and integrated to filterscanner to use the pruned bitset from FG datamap.
FG Query flow as follows.
1.The user can add FG datamap to any table and implement there interfaces.
2. Any filter query which hits the table with datamap will call prune method of FGdatamap.
3. The prune method of FGDatamap return list FineGrainBlocklet , these blocklets contain the information of block, blocklet, page and rowids information as well.
4. The pruned blocklets are internally wriitten to file and returns only the block , blocklet and filepath information as part of Splits.
5. Based on the splits scanrdd schedule the tasks.
6. In filterscanner we check the datamapwriterpath from split and reNoteads the bitset if exists. And pass this bitset as input to it.

This closes #1471


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

Branch: refs/heads/carbonstore-rebase4
Commit: e992013a135a2affeef2f5574c739e10d12538cc
Parents: ef43759
Author: ravipesala <ra...@gmail.com>
Authored: Wed Nov 15 19:48:40 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 00:30:20 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/datamap/DataMapMeta.java    |   8 +-
 .../core/datamap/DataMapStoreManager.java       |  30 +-
 .../carbondata/core/datamap/DataMapType.java    |  21 +
 .../carbondata/core/datamap/TableDataMap.java   |  31 +-
 .../core/datamap/dev/AbstractDataMapWriter.java | 110 +++++
 .../core/datamap/dev/BlockletSerializer.java    |  57 +++
 .../carbondata/core/datamap/dev/DataMap.java    |   4 +-
 .../core/datamap/dev/DataMapFactory.java        |  14 +-
 .../core/datamap/dev/DataMapWriter.java         |  57 ---
 .../cgdatamap/AbstractCoarseGrainDataMap.java   |  24 +
 .../AbstractCoarseGrainDataMapFactory.java      |  34 ++
 .../dev/fgdatamap/AbstractFineGrainDataMap.java |  24 +
 .../AbstractFineGrainDataMapFactory.java        |  38 ++
 .../carbondata/core/datastore/DataRefNode.java  |   6 +
 .../core/datastore/block/TableBlockInfo.java    |  10 +
 .../impl/btree/AbstractBTreeLeafNode.java       |   5 +
 .../datastore/impl/btree/BTreeNonLeafNode.java  |   5 +
 .../carbondata/core/indexstore/Blocklet.java    |  30 +-
 .../indexstore/BlockletDataMapIndexStore.java   |   6 -
 .../core/indexstore/BlockletDetailsFetcher.java |   8 +
 .../core/indexstore/ExtendedBlocklet.java       |  17 +
 .../core/indexstore/FineGrainBlocklet.java      | 120 +++++
 .../blockletindex/BlockletDataMap.java          |  15 +-
 .../blockletindex/BlockletDataMapFactory.java   |  55 ++-
 .../blockletindex/BlockletDataRefNode.java      |  27 +-
 .../indexstore/blockletindex/IndexWrapper.java  |  18 +
 .../core/indexstore/schema/FilterType.java      |  24 -
 .../executer/ExcludeFilterExecuterImpl.java     |   3 +
 .../executer/IncludeFilterExecuterImpl.java     |   3 +
 .../scanner/impl/BlockletFilterScanner.java     |   2 +
 .../apache/carbondata/core/util/CarbonUtil.java |  98 +++++
 .../datamap/examples/MinMaxDataMap.java         |  32 +-
 .../datamap/examples/MinMaxDataMapFactory.java  |  49 ++-
 .../datamap/examples/MinMaxDataWriter.java      |  36 +-
 .../examples/MinMaxIndexBlockDetails.java       |  13 -
 .../carbondata/hadoop/CarbonInputSplit.java     |  21 +-
 .../hadoop/api/CarbonTableInputFormat.java      |  17 +-
 .../testsuite/datamap/CGDataMapTestCase.scala   | 361 +++++++++++++++
 .../testsuite/datamap/DataMapWriterSuite.scala  |  43 +-
 .../testsuite/datamap/FGDataMapTestCase.scala   | 440 +++++++++++++++++++
 .../TestInsertAndOtherCommandConcurrent.scala   |  21 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   7 +-
 .../TestStreamingTableOperation.scala           |   5 +-
 .../datamap/DataMapWriterListener.java          |  57 ++-
 .../store/CarbonFactDataHandlerModel.java       |  10 +-
 .../store/writer/AbstractFactDataWriter.java    | 128 +-----
 46 files changed, 1762 insertions(+), 382 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
index 7746acf..dd15ccb 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapMeta.java
@@ -19,15 +19,15 @@ package org.apache.carbondata.core.datamap;
 
 import java.util.List;
 
-import org.apache.carbondata.core.indexstore.schema.FilterType;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
 
 public class DataMapMeta {
 
   private List<String> indexedColumns;
 
-  private FilterType optimizedOperation;
+  private List<ExpressionType> optimizedOperation;
 
-  public DataMapMeta(List<String> indexedColumns, FilterType optimizedOperation) {
+  public DataMapMeta(List<String> indexedColumns, List<ExpressionType> optimizedOperation) {
     this.indexedColumns = indexedColumns;
     this.optimizedOperation = optimizedOperation;
   }
@@ -36,7 +36,7 @@ public class DataMapMeta {
     return indexedColumns;
   }
 
-  public FilterType getOptimizedOperation() {
+  public List<ExpressionType> getOptimizedOperation() {
     return optimizedOperation;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
index 90e5fff..8d80b4d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
@@ -56,7 +56,22 @@ public final class DataMapStoreManager {
   }
 
   public List<TableDataMap> getAllDataMap(AbsoluteTableIdentifier identifier) {
-    return allDataMaps.get(identifier.uniqueName());
+    return allDataMaps.get(identifier.getCarbonTableIdentifier().getTableUniqueName());
+  }
+
+  // TODO its a temporary method till chooser is implemented
+  public TableDataMap chooseDataMap(AbsoluteTableIdentifier identifier) {
+    List<TableDataMap> tableDataMaps = getAllDataMap(identifier);
+    if (tableDataMaps != null && tableDataMaps.size() > 0) {
+      for (TableDataMap dataMap: tableDataMaps) {
+        if (!dataMap.getDataMapName().equalsIgnoreCase(BlockletDataMap.NAME)) {
+          return dataMap;
+        }
+      }
+      return tableDataMaps.get(0);
+    } else {
+      return getDataMap(identifier, BlockletDataMap.NAME, BlockletDataMapFactory.class.getName());
+    }
   }
 
   /**
@@ -68,7 +83,7 @@ public final class DataMapStoreManager {
    */
   public TableDataMap getDataMap(AbsoluteTableIdentifier identifier,
       String dataMapName, String factoryClass) {
-    String table = identifier.uniqueName();
+    String table = identifier.getCarbonTableIdentifier().getTableUniqueName();
     List<TableDataMap> tableDataMaps = allDataMaps.get(table);
     TableDataMap dataMap;
     if (tableDataMaps == null) {
@@ -96,7 +111,7 @@ public final class DataMapStoreManager {
    */
   public TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
       String factoryClassName, String dataMapName) {
-    String table = identifier.uniqueName();
+    String table = identifier.getCarbonTableIdentifier().getTableUniqueName();
     // Just update the segmentRefreshMap with the table if not added.
     getTableSegmentRefresher(identifier);
     List<TableDataMap> tableDataMaps = allDataMaps.get(table);
@@ -149,7 +164,9 @@ public final class DataMapStoreManager {
    * @param identifier Table identifier
    */
   public void clearDataMaps(AbsoluteTableIdentifier identifier) {
-    List<TableDataMap> tableDataMaps = allDataMaps.get(identifier.uniqueName());
+    String tableUniqueName = identifier.getCarbonTableIdentifier().getTableUniqueName();
+    List<TableDataMap> tableDataMaps =
+        allDataMaps.get(tableUniqueName);
     segmentRefreshMap.remove(identifier.uniqueName());
     if (tableDataMaps != null) {
       for (TableDataMap tableDataMap: tableDataMaps) {
@@ -158,7 +175,7 @@ public final class DataMapStoreManager {
           break;
         }
       }
-      allDataMaps.remove(identifier.uniqueName());
+      allDataMaps.remove(tableUniqueName);
     }
   }
 
@@ -167,7 +184,8 @@ public final class DataMapStoreManager {
    * @param identifier Table identifier
    */
   public void clearDataMap(AbsoluteTableIdentifier identifier, String dataMapName) {
-    List<TableDataMap> tableDataMaps = allDataMaps.get(identifier.uniqueName());
+    List<TableDataMap> tableDataMaps =
+        allDataMaps.get(identifier.getCarbonTableIdentifier().getTableUniqueName());
     if (tableDataMaps != null) {
       int i = 0;
       for (TableDataMap tableDataMap: tableDataMaps) {

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index a841f37..eed650e3 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -21,13 +21,16 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datamap.dev.BlockletSerializer;
 import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -77,10 +80,15 @@ public final class TableDataMap extends OperationEventListener {
     SegmentProperties segmentProperties;
     for (Segment segment : segments) {
       List<Blocklet> pruneBlocklets = new ArrayList<>();
-      List<DataMap> dataMaps = dataMapFactory.getDataMaps(segment);
-      segmentProperties = segmentPropertiesFetcher.getSegmentProperties(segment.getSegmentNo());
-      for (DataMap dataMap : dataMaps) {
-        pruneBlocklets.addAll(dataMap.prune(filterExp, segmentProperties, partitions));
+      // if filter is not passed then return all the blocklets
+      if (filterExp == null) {
+        pruneBlocklets = blockletDetailsFetcher.getAllBlocklets(segment, partitions);
+      } else {
+        List<DataMap> dataMaps = dataMapFactory.getDataMaps(segment);
+        segmentProperties = segmentPropertiesFetcher.getSegmentProperties(segment);
+        for (DataMap dataMap : dataMaps) {
+          pruneBlocklets.addAll(dataMap.prune(filterExp, segmentProperties, partitions));
+        }
       }
       blocklets.addAll(addSegmentId(blockletDetailsFetcher
           .getExtendedBlocklets(pruneBlocklets, segment), segment.getSegmentNo()));
@@ -138,10 +146,21 @@ public final class TableDataMap extends OperationEventListener {
               segmentPropertiesFetcher.getSegmentProperties(distributable.getSegmentId()),
               partitions));
     }
-    for (Blocklet blocklet: blocklets) {
+    BlockletSerializer serializer = new BlockletSerializer();
+    String writePath =
+        identifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR + dataMapName;
+    if (dataMapFactory.getDataMapType() == DataMapType.FG) {
+      FileFactory.mkdirs(writePath, FileFactory.getFileType(writePath));
+    }
+    for (Blocklet blocklet : blocklets) {
       ExtendedBlocklet detailedBlocklet =
           blockletDetailsFetcher.getExtendedBlocklet(blocklet, distributable.getSegment());
-      detailedBlocklet.setSegmentId(distributable.getSegment().getSegmentNo());
+      if (dataMapFactory.getDataMapType() == DataMapType.FG) {
+        String blockletwritePath =
+            writePath + CarbonCommonConstants.FILE_SEPARATOR + System.nanoTime();
+        detailedBlocklet.setDataMapWriterPath(blockletwritePath);
+        serializer.serializeBlocklet((FineGrainBlocklet) blocklet, blockletwritePath);
+      }detailedBlocklet.setSegmentId(distributable.getSegment().getSegmentNo());
       detailedBlocklets.add(detailedBlocklet);
     }
     return detailedBlocklets;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
new file mode 100644
index 0000000..bcc9bad
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+/**
+ * Data Map writer
+ */
+public abstract class AbstractDataMapWriter {
+
+  protected AbsoluteTableIdentifier identifier;
+
+  protected String segmentId;
+
+  protected String writeDirectoryPath;
+
+  public AbstractDataMapWriter(AbsoluteTableIdentifier identifier, String segmentId,
+      String writeDirectoryPath) {
+    this.identifier = identifier;
+    this.segmentId = segmentId;
+    this.writeDirectoryPath = writeDirectoryPath;
+  }
+
+  /**
+   * Start of new block notification.
+   *
+   * @param blockId file name of the carbondata file
+   */
+  public abstract void onBlockStart(String blockId);
+
+  /**
+   * End of block notification
+   */
+  public abstract void onBlockEnd(String blockId);
+
+  /**
+   * Start of new blocklet notification.
+   *
+   * @param blockletId sequence number of blocklet in the block
+   */
+  public abstract void onBlockletStart(int blockletId);
+
+  /**
+   * End of blocklet notification
+   *
+   * @param blockletId sequence number of blocklet in the block
+   */
+  public abstract void onBlockletEnd(int blockletId);
+
+  /**
+   * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
+   * DataMapMeta returned in DataMapFactory.
+   * Implementation should copy the content of `pages` as needed, because `pages` memory
+   * may be freed after this method returns, if using unsafe column page.
+   */
+  public abstract void onPageAdded(int blockletId, int pageId, ColumnPage[] pages);
+
+  /**
+   * This is called during closing of writer.So after this call no more data will be sent to this
+   * class.
+   */
+  public abstract void finish() throws IOException;
+
+  /**
+   * It copies the file from temp folder to actual folder
+   *
+   * @param dataMapFile
+   * @throws IOException
+   */
+  protected void commitFile(String dataMapFile) throws IOException {
+    if (!dataMapFile.startsWith(writeDirectoryPath)) {
+      throw new UnsupportedOperationException(
+          "Datamap file " + dataMapFile + " is not written in provided directory path "
+              + writeDirectoryPath);
+    }
+    String dataMapFileName =
+        dataMapFile.substring(writeDirectoryPath.length(), dataMapFile.length());
+    String carbonFilePath = dataMapFileName.substring(0, dataMapFileName.lastIndexOf("/"));
+    String segmentPath = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
+    if (carbonFilePath.length() > 0) {
+      carbonFilePath = segmentPath + carbonFilePath;
+      FileFactory.mkdirs(carbonFilePath, FileFactory.getFileType(carbonFilePath));
+    } else {
+      carbonFilePath = segmentPath;
+    }
+    CarbonUtil.copyCarbonDataFileToCarbonStorePath(dataMapFile, carbonFilePath, 0);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
new file mode 100644
index 0000000..3d4c717
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/BlockletSerializer.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
+
+public class BlockletSerializer {
+
+  /**
+   * Serialize and write blocklet to the file.
+   * @param grainBlocklet
+   * @param writePath
+   * @throws IOException
+   */
+  public void serializeBlocklet(FineGrainBlocklet grainBlocklet, String writePath)
+      throws IOException {
+    DataOutputStream dataOutputStream =
+        FileFactory.getDataOutputStream(writePath, FileFactory.getFileType(writePath));
+    grainBlocklet.write(dataOutputStream);
+    dataOutputStream.close();
+  }
+
+  /**
+   * Read data from filepath and deserialize blocklet.
+   * @param writePath
+   * @return
+   * @throws IOException
+   */
+  public FineGrainBlocklet deserializeBlocklet(String writePath) throws IOException {
+    DataInputStream inputStream =
+        FileFactory.getDataInputStream(writePath, FileFactory.getFileType(writePath));
+    FineGrainBlocklet blocklet = new FineGrainBlocklet();
+    blocklet.readFields(inputStream);
+    inputStream.close();
+    return blocklet;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
index 434b371..4a68286 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMap.java
@@ -28,7 +28,7 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 /**
  * Datamap is an entity which can store and retrieve index data.
  */
-public interface DataMap {
+public interface DataMap<T extends Blocklet> {
 
   /**
    * It is called to load the data map to memory or to initialize it.
@@ -42,7 +42,7 @@ public interface DataMap {
    * @param filterExp
    * @return
    */
-  List<Blocklet> prune(FilterResolverIntf filterExp, List<PartitionSpec> partitions);
+  List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties, List<PartitionSpec> partitions);
 
   // TODO Move this method to Abstract class
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index 40cd436..df5670d 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -22,13 +22,14 @@ import java.util.List;
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.DataMapType;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.events.Event;
 
 /**
  * Interface for datamap factory, it is responsible for creating the datamap.
  */
-public interface DataMapFactory {
+public interface DataMapFactory<T extends DataMap> {
 
   /**
    * Initialization of Datamap factory with the identifier and datamap name
@@ -38,17 +39,17 @@ public interface DataMapFactory {
   /**
    * Return a new write for this datamap
    */
-  DataMapWriter createWriter(Segment segment);
+  AbstractDataMapWriter createWriter(Segment segment);
 
   /**
    * Get the datamap for segmentid
    */
-  List<DataMap> getDataMaps(Segment segment) throws IOException;
+  List<T> getDataMaps(Segment segment) throws IOException;
 
   /**
    * Get datamaps for distributable object.
    */
-  List<DataMap> getDataMaps(DataMapDistributable distributable) throws IOException;
+  List<T> getDataMaps(DataMapDistributable distributable) throws IOException;
 
   /**
    * Get all distributable objects of a segmentid
@@ -76,4 +77,9 @@ public interface DataMapFactory {
    * Return metadata of this datamap
    */
   DataMapMeta getMeta();
+
+  /**
+   *  Type of datamap whether it is FG or CG
+   */
+  DataMapType getDataMapType();
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
deleted file mode 100644
index 413eaa5..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapWriter.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datamap.dev;
-
-import org.apache.carbondata.core.datastore.page.ColumnPage;
-
-/**
- * Data Map writer
- */
-public interface DataMapWriter {
-
-  /**
-   *  Start of new block notification.
-   *  @param blockId file name of the carbondata file
-   */
-  void onBlockStart(String blockId, String blockPath);
-
-  /**
-   * End of block notification
-   */
-  void onBlockEnd(String blockId);
-
-  /**
-   * Start of new blocklet notification.
-   * @param blockletId sequence number of blocklet in the block
-   */
-  void onBlockletStart(int blockletId);
-
-  /**
-   * End of blocklet notification
-   * @param blockletId sequence number of blocklet in the block
-   */
-  void onBlockletEnd(int blockletId);
-  /**
-   * Add the column pages row to the datamap, order of pages is same as `indexColumns` in
-   * DataMapMeta returned in DataMapFactory.
-   *
-   * Implementation should copy the content of `pages` as needed, because `pages` memory
-   * may be freed after this method returns, if using unsafe column page.
-   */
-  void onPageAdded(int blockletId, int pageId, ColumnPage[] pages);
-
-}

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMapFactory.java
new file mode 100644
index 0000000..9789992
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/cgdatamap/AbstractCoarseGrainDataMapFactory.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datamap.dev.cgdatamap;
+
+import org.apache.carbondata.core.datamap.DataMapType;
+import org.apache.carbondata.core.datamap.dev.DataMapFactory;
+
+/**
+ *  1. Any filter query which hits the table with datamap will call prune method of CGdatamap.
+ *  2. The prune method of CGDatamap return list Blocklet , these blocklets contain the
+ *     information of block and blocklet.
+ *  3. Based on the splits scanrdd schedule the tasks.
+ */
+public abstract class AbstractCoarseGrainDataMapFactory
+    implements DataMapFactory<AbstractCoarseGrainDataMap> {
+
+  @Override public DataMapType getDataMapType() {
+    return DataMapType.CG;
+  }
+}

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

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
index 273f833..df0896a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/DataRefNode.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
  * Interface data block reference
@@ -119,4 +120,9 @@ public interface DataRefNode {
    */
   MeasureRawColumnChunk readMeasureChunk(FileReader fileReader, int columnIndex) throws IOException;
 
+  /**
+   * Return the indexed data if it has any from disk which was stored by FG datamap.
+   * @return
+   */
+  BitSetGroup getIndexedData();
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index b27b5fc..a7bfdba 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -96,6 +96,8 @@ public class TableBlockInfo implements Distributable, Serializable {
 
   private BlockletDetailInfo detailInfo;
 
+  private String dataMapWriterPath;
+
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
       String[] locations, long blockLength, ColumnarFormatVersion version,
       String[] deletedDeltaFilePath) {
@@ -424,4 +426,12 @@ public class TableBlockInfo implements Distributable, Serializable {
   public void setDataBlockFromOldStore(boolean dataBlockFromOldStore) {
     isDataBlockFromOldStore = dataBlockFromOldStore;
   }
+
+  public String getDataMapWriterPath() {
+    return dataMapWriterPath;
+  }
+
+  public void setDataMapWriterPath(String dataMapWriterPath) {
+    this.dataMapWriterPath = dataMapWriterPath;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
index fe4cf83..f5a751b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/AbstractBTreeLeafNode.java
@@ -23,6 +23,7 @@ import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
  * Non leaf node abstract class
@@ -222,4 +223,8 @@ public abstract class AbstractBTreeLeafNode implements BTreeNode {
   public int getPageRowCount(int pageNumber) {
     throw new UnsupportedOperationException("Unsupported operation");
   }
+
+  @Override public BitSetGroup getIndexedData() {
+    return null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
index c200f8d..a6eb695 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/btree/BTreeNonLeafNode.java
@@ -25,6 +25,7 @@ import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
  * No leaf node of a b+tree class which will keep the matadata(start key) of the
@@ -227,6 +228,10 @@ public class BTreeNonLeafNode implements BTreeNode {
     throw new UnsupportedOperationException("Unsupported operation");
   }
 
+  public BitSetGroup getIndexedData() {
+    return null;
+  }
+
   /**
    * number of pages in blocklet
    * @return

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
index d84f3f6..c731e07 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/Blocklet.java
@@ -16,28 +16,46 @@
  */
 package org.apache.carbondata.core.indexstore;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 
+import org.apache.carbondata.core.metadata.schema.table.Writable;
+
 /**
  * Blocklet
  */
-public class Blocklet implements Serializable {
+public class Blocklet implements Writable,Serializable {
 
-  private String path;
+  private String blockId;
 
   private String blockletId;
 
-  public Blocklet(String path, String blockletId) {
-    this.path = path;
+  public Blocklet(String blockId, String blockletId) {
+    this.blockId = blockId;
     this.blockletId = blockletId;
   }
 
-  public String getPath() {
-    return path;
+  // For serialization purpose
+  public Blocklet() {
   }
 
   public String getBlockletId() {
     return blockletId;
   }
 
+  public String getBlockId() {
+    return blockId;
+  }
+
+  @Override public void write(DataOutput out) throws IOException {
+    out.writeUTF(blockId);
+    out.writeUTF(blockletId);
+  }
+
+  @Override public void readFields(DataInput in) throws IOException {
+    blockId = in.readUTF();
+    blockletId = in.readUTF();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
index c6073d5..defaf39 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDataMapIndexStore.java
@@ -22,7 +22,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutorService;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -139,7 +138,6 @@ public class BlockletDataMapIndexStore
       List<TableBlockIndexUniqueIdentifier> tableSegmentUniqueIdentifiers) throws IOException {
     List<BlockletDataMap> blockletDataMaps = new ArrayList<>(tableSegmentUniqueIdentifiers.size());
     List<TableBlockIndexUniqueIdentifier> missedIdentifiers = new ArrayList<>();
-    ExecutorService service = null;
     // Get the datamaps for each indexfile from cache.
     try {
       for (TableBlockIndexUniqueIdentifier identifier : tableSegmentUniqueIdentifiers) {
@@ -165,10 +163,6 @@ public class BlockletDataMapIndexStore
         dataMap.clear();
       }
       throw new IOException("Problem in loading segment blocks.", e);
-    } finally {
-      if (service != null) {
-        service.shutdownNow();
-      }
     }
     return blockletDataMaps;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
index b4d6db2..5a5fc1e 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
@@ -46,4 +46,12 @@ public interface BlockletDetailsFetcher {
    * @throws IOException
    */
   ExtendedBlocklet getExtendedBlocklet(Blocklet blocklet, Segment segment) throws IOException;
+
+  /**
+   * Get all the blocklets in a segment
+   *
+   * @param segment
+   * @return
+   */
+  List<Blocklet> getAllBlocklets(Segment segment, List<String> partitions) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
index d1bfa35..58a9344 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
@@ -29,8 +29,13 @@ public class ExtendedBlocklet extends Blocklet {
 
   private String[] location;
 
+  private String path;
+
+  private String dataMapWriterPath;
+
   public ExtendedBlocklet(String path, String blockletId) {
     super(path, blockletId);
+    this.path = path;
   }
 
   public BlockletDetailInfo getDetailInfo() {
@@ -60,4 +65,16 @@ public class ExtendedBlocklet extends Blocklet {
   public void setSegmentId(String segmentId) {
     this.segmentId = segmentId;
   }
+
+  public String getPath() {
+    return path;
+  }
+
+  public String getDataMapWriterPath() {
+    return dataMapWriterPath;
+  }
+
+  public void setDataMapWriterPath(String dataMapWriterPath) {
+    this.dataMapWriterPath = dataMapWriterPath;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java b/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
new file mode 100644
index 0000000..266120e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/FineGrainBlocklet.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.indexstore;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
+import org.apache.carbondata.core.metadata.schema.table.Writable;
+import org.apache.carbondata.core.util.BitSetGroup;
+
+/**
+ * FineGrainBlocklet
+ */
+public class FineGrainBlocklet extends Blocklet implements Serializable {
+
+  private List<Page> pages;
+
+  public FineGrainBlocklet(String blockId, String blockletId, List<Page> pages) {
+    super(blockId, blockletId);
+    this.pages = pages;
+  }
+
+  // For serialization purpose
+  public FineGrainBlocklet() {
+
+  }
+
+  public List<Page> getPages() {
+    return pages;
+  }
+
+  public static class Page implements Writable,Serializable {
+
+    private int pageId;
+
+    private int[] rowId;
+
+    public BitSet getBitSet() {
+      BitSet bitSet =
+          new BitSet(CarbonV3DataFormatConstants.NUMBER_OF_ROWS_PER_BLOCKLET_COLUMN_PAGE_DEFAULT);
+      for (int row : rowId) {
+        bitSet.set(row);
+      }
+      return bitSet;
+    }
+
+    @Override public void write(DataOutput out) throws IOException {
+      out.writeInt(pageId);
+      out.writeInt(rowId.length);
+      for (int i = 0; i < rowId.length; i++) {
+        out.writeInt(rowId[i]);
+      }
+    }
+
+    @Override public void readFields(DataInput in) throws IOException {
+      pageId = in.readInt();
+      int length = in.readInt();
+      rowId = new int[length];
+      for (int i = 0; i < length; i++) {
+        rowId[i] = in.readInt();
+      }
+    }
+
+    public void setPageId(int pageId) {
+      this.pageId = pageId;
+    }
+
+    public void setRowId(int[] rowId) {
+      this.rowId = rowId;
+    }
+  }
+
+  public BitSetGroup getBitSetGroup(int numberOfPages) {
+    BitSetGroup bitSetGroup = new BitSetGroup(numberOfPages);
+    for (int i = 0; i < pages.size(); i++) {
+      bitSetGroup.setBitSet(pages.get(i).getBitSet(), pages.get(i).pageId);
+    }
+    return bitSetGroup;
+  }
+
+  @Override public void write(DataOutput out) throws IOException {
+    super.write(out);
+    int size = pages.size();
+    out.writeInt(size);
+    for (Page page : pages) {
+      page.write(out);
+    }
+  }
+
+  @Override public void readFields(DataInput in) throws IOException {
+    super.readFields(in);
+    int size = in.readInt();
+    pages = new ArrayList<>(size);
+    for (int i = 0; i < size; i++) {
+      Page page = new Page();
+      page.readFields(in);
+      pages.add(page);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index 4bd6ae7..ef1bd33 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -34,8 +34,8 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.cache.Cacheable;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMap;
 import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
 import org.apache.carbondata.core.datastore.IndexKey;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
@@ -75,7 +75,7 @@ import org.xerial.snappy.Snappy;
 /**
  * Datamap implementation for blocklet.
  */
-public class BlockletDataMap implements DataMap, Cacheable {
+public class BlockletDataMap extends AbstractCoarseGrainDataMap implements Cacheable {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(BlockletDataMap.class.getName());
@@ -614,9 +614,9 @@ public class BlockletDataMap implements DataMap, Cacheable {
         FilterUtil.getFilterExecuterTree(filterExp, segmentProperties, null);
     for (int i = 0; i < unsafeMemorySummaryDMStore.getRowCount(); i++) {
       DataMapRow unsafeRow = unsafeMemorySummaryDMStore.getUnsafeRow(i);
-      boolean isScanRequired = FilterExpressionProcessor
-          .isScanRequired(filterExecuter, getMinMaxValue(unsafeRow, TASK_MAX_VALUES_INDEX),
-              getMinMaxValue(unsafeRow, TASK_MIN_VALUES_INDEX));
+      boolean isScanRequired = FilterExpressionProcessor.isScanRequired(
+          filterExecuter, getMinMaxValue(unsafeRow, TASK_MAX_VALUES_INDEX),
+          getMinMaxValue(unsafeRow, TASK_MIN_VALUES_INDEX));
       if (isScanRequired) {
         return true;
       }
@@ -656,7 +656,6 @@ public class BlockletDataMap implements DataMap, Cacheable {
         startIndex++;
       }
     }
-
     return blocklets;
   }
 
@@ -966,4 +965,8 @@ public class BlockletDataMap implements DataMap, Cacheable {
     return memoryUsed;
   }
 
+  public SegmentProperties getSegmentProperties() {
+    return segmentProperties;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index 89e61d2..5ca3ac5 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -28,9 +28,10 @@ import org.apache.carbondata.core.cache.CacheType;
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
 import org.apache.carbondata.core.datamap.Segment;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
 import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMapFactory;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -55,7 +56,8 @@ import org.apache.hadoop.fs.RemoteIterator;
 /**
  * Table map for blocklet
  */
-public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFetcher,
+public class BlockletDataMapFactory extends AbstractCoarseGrainDataMapFactory
+    implements BlockletDetailsFetcher,
     SegmentPropertiesFetcher {
 
   private AbsoluteTableIdentifier identifier;
@@ -63,10 +65,7 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
   // segmentId -> list of index file
   private Map<String, List<TableBlockIndexUniqueIdentifier>> segmentMap = new HashMap<>();
 
-  // segmentId -> SegmentProperties.
-  private Map<String, SegmentProperties> segmentPropertiesMap = new HashMap<>();
-
-  private Cache<TableBlockIndexUniqueIdentifier, DataMap> cache;
+  private Cache<TableBlockIndexUniqueIdentifier, AbstractCoarseGrainDataMap> cache;
 
   @Override
   public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
@@ -152,16 +151,17 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
 
   private ExtendedBlocklet getExtendedBlocklet(List<TableBlockIndexUniqueIdentifier> identifiers,
       Blocklet blocklet) throws IOException {
-    String carbonIndexFileName = CarbonTablePath.getCarbonIndexFileName(blocklet.getPath());
+    String carbonIndexFileName = CarbonTablePath.getCarbonIndexFileName(blocklet.getBlockId());
     for (TableBlockIndexUniqueIdentifier identifier : identifiers) {
       if (identifier.getIndexFilePath().equals(carbonIndexFileName)) {
         DataMap dataMap = cache.get(identifier);
         return ((BlockletDataMap) dataMap).getDetailedBlocklet(blocklet.getBlockletId());
       }
     }
-    throw new IOException("Blocklet with blockid " + blocklet.getPath() + " not found ");
+    throw new IOException("Blocklet with blockid " + blocklet.getBlockletId() + " not found ");
   }
 
+
   @Override
   public List<DataMapDistributable> toDistributable(Segment segment) {
     List<DataMapDistributable> distributables = new ArrayList<>();
@@ -226,7 +226,8 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
   }
 
   @Override
-  public List<DataMap> getDataMaps(DataMapDistributable distributable) throws IOException {
+  public List<AbstractCoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
+      throws IOException {
     BlockletDataMapDistributable mapDistributable = (BlockletDataMapDistributable) distributable;
     List<TableBlockIndexUniqueIdentifier> identifiers = new ArrayList<>();
     Path indexPath = new Path(mapDistributable.getFilePath());
@@ -246,7 +247,7 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
                 segmentNo));
       }
     }
-    List<DataMap> dataMaps;
+    List<AbstractCoarseGrainDataMap> dataMaps;
     try {
       dataMaps = cache.getAll(identifiers);
     } catch (IOException e) {
@@ -262,23 +263,21 @@ public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFe
   }
 
   @Override public SegmentProperties getSegmentProperties(String segmentId) throws IOException {
-    SegmentProperties segmentProperties = segmentPropertiesMap.get(segmentId);
-    if (segmentProperties == null) {
-      int[] columnCardinality;
-      List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
-          getTableBlockIndexUniqueIdentifiers(segmentId);
-      DataFileFooterConverter fileFooterConverter = new DataFileFooterConverter();
-      List<DataFileFooter> indexInfo =
-          fileFooterConverter.getIndexInfo(tableBlockIndexUniqueIdentifiers.get(0).getFilePath());
-      for (DataFileFooter fileFooter : indexInfo) {
-        List<ColumnSchema> columnInTable = fileFooter.getColumnInTable();
-        if (segmentProperties == null) {
-          columnCardinality = fileFooter.getSegmentInfo().getColumnCardinality();
-          segmentProperties = new SegmentProperties(columnInTable, columnCardinality);
-        }
-      }
-      segmentPropertiesMap.put(segmentId, segmentProperties);
+    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segmentId);
+    assert (dataMaps.size() > 0);
+    AbstractCoarseGrainDataMap coarseGrainDataMap = dataMaps.get(0);
+    assert (coarseGrainDataMap instanceof BlockletDataMap);
+    BlockletDataMap dataMap = (BlockletDataMap) coarseGrainDataMap;
+    return dataMap.getSegmentProperties();
+  }
+
+  @Override public List<Blocklet> getAllBlocklets(String segmentId, List<String> partitions)
+      throws IOException {
+    List<Blocklet> blocklets = new ArrayList<>();
+    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segmentId);
+    for (AbstractCoarseGrainDataMap dataMap : dataMaps) {
+      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segmentId), partitions));
     }
-    return segmentProperties;
+    return blocklets;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
index 5cd59cb..b8fd6ff 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.constants.CarbonVersionConstants;
+import org.apache.carbondata.core.datamap.dev.BlockletSerializer;
 import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
@@ -30,7 +31,9 @@ import org.apache.carbondata.core.datastore.chunk.reader.CarbonDataReaderFactory
 import org.apache.carbondata.core.datastore.chunk.reader.DimensionColumnChunkReader;
 import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
+import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
 
 /**
@@ -44,6 +47,8 @@ public class BlockletDataRefNode implements DataRefNode {
 
   private int[] dimensionLens;
 
+  private BlockletSerializer blockletSerializer;
+
   BlockletDataRefNode(List<TableBlockInfo> blockInfos, int index, int[] dimensionLens) {
     this.blockInfos = blockInfos;
     // Update row count and page count to blocklet info
@@ -76,6 +81,7 @@ public class BlockletDataRefNode implements DataRefNode {
     }
     this.index = index;
     this.dimensionLens = dimensionLens;
+    this.blockletSerializer = new BlockletSerializer();
   }
 
   @Override public DataRefNode getNextDataRefNode() {
@@ -209,11 +215,28 @@ public class BlockletDataRefNode implements DataRefNode {
     }
   }
 
-  @Override public int numberOfPages() {
+  @Override
+  public int numberOfPages() {
     return blockInfos.get(index).getDetailInfo().getPagesCount();
   }
 
-  @Override public int getPageRowCount(int pageNumber) {
+  @Override
+  public BitSetGroup getIndexedData() {
+    String dataMapWriterPath = blockInfos.get(index).getDataMapWriterPath();
+    if (dataMapWriterPath != null) {
+      try {
+        FineGrainBlocklet blocklet = blockletSerializer.deserializeBlocklet(dataMapWriterPath);
+        return blocklet.getBitSetGroup(numberOfPages());
+      } catch (IOException e) {
+        return null;
+      }
+    } else {
+      return null;
+    }
+  }
+
+  @Override
+  public int getPageRowCount(int pageNumber) {
     return blockInfos.get(index).getDetailInfo().getBlockletInfo()
         .getNumberOfRowsPerPage()[pageNumber];
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
index a30f64c..95232e5 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/IndexWrapper.java
@@ -21,6 +21,8 @@ import java.util.List;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 
 /**
@@ -29,7 +31,10 @@ import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
  */
 public class IndexWrapper extends AbstractIndex {
 
+  private List<TableBlockInfo> blockInfos;
+
   public IndexWrapper(List<TableBlockInfo> blockInfos) {
+    this.blockInfos = blockInfos;
     segmentProperties = new SegmentProperties(blockInfos.get(0).getDetailInfo().getColumnSchemas(),
         blockInfos.get(0).getDetailInfo().getDimLens());
     dataRefNode = new BlockletDataRefNode(blockInfos, 0,
@@ -38,4 +43,17 @@ public class IndexWrapper extends AbstractIndex {
 
   @Override public void buildIndex(List<DataFileFooter> footerList) {
   }
+
+  @Override public void clear() {
+    super.clear();
+    if (blockInfos != null) {
+      for (TableBlockInfo blockInfo : blockInfos) {
+        String dataMapWriterPath = blockInfo.getDataMapWriterPath();
+        if (dataMapWriterPath != null) {
+          CarbonFile file = FileFactory.getCarbonFile(dataMapWriterPath);
+          FileFactory.deleteAllCarbonFilesOfDir(file);
+        }
+      }
+    }
+  }
 }

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
index fe983a5..a608c8e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImpl.java
@@ -314,6 +314,9 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
   private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnPage dimensionColumnPage,
       BitSetGroup prvBitSetGroup, int pageNumber) {
     BitSet prvPageBitSet = prvBitSetGroup.getBitSet(pageNumber);
+    if (prvPageBitSet == null || prvPageBitSet.isEmpty()) {
+      return prvPageBitSet;
+    }
     BitSet bitSet = new BitSet();
     bitSet.or(prvPageBitSet);
     byte[][] filterKeys = dimColumnExecuterInfo.getExcludeFilterKeys();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
index 05328f3..516ed41 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImpl.java
@@ -334,6 +334,9 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
   private BitSet getFilteredIndexesUisngPrvBitset(DimensionColumnPage dimensionColumnPage,
       BitSetGroup prvBitSetGroup, int pageNumber, int numberOfRows) {
     BitSet prvPageBitSet = prvBitSetGroup.getBitSet(pageNumber);
+    if (prvPageBitSet == null || prvPageBitSet.isEmpty()) {
+      return prvPageBitSet;
+    }
     BitSet bitSet = new BitSet(numberOfRows);
     byte[][] filterKeys = dimColumnExecuterInfo.getFilterKeys();
     int compareResult = 0;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
index 1c73d63..033c3dd 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/scanner/impl/BlockletFilterScanner.java
@@ -163,6 +163,8 @@ public class BlockletFilterScanner extends BlockletFullScanner {
         .get(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM);
     totalBlockletStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM,
         totalBlockletStatistic.getCount() + 1);
+    // set the indexed data if it has any during fgdatamap pruning.
+    rawBlockletColumnChunks.setBitSetGroup(rawBlockletColumnChunks.getDataBlock().getIndexedData());
     // apply filter on actual data, for each page
     BitSetGroup bitSetGroup = this.filterExecuter.applyFilter(rawBlockletColumnChunks,
         useBitSetPipeLine);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 0adc3a5..d531b5c 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -22,6 +22,7 @@ import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
 import java.io.DataInputStream;
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
@@ -56,6 +57,7 @@ import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
 import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
+import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
@@ -105,6 +107,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.thrift.TBase;
 import org.apache.thrift.TException;
@@ -137,6 +140,13 @@ public final class CarbonUtil {
 
   private static final Configuration conf = new Configuration(true);
 
+  /**
+   * dfs.bytes-per-checksum
+   * HDFS checksum length, block size for a file should be exactly divisible
+   * by this value
+   */
+  private static final int HDFS_CHECKSUM_LENGTH = 512;
+
   private CarbonUtil() {
 
   }
@@ -2429,6 +2439,94 @@ public final class CarbonUtil {
     return Base64.decodeBase64(objectString.getBytes(CarbonCommonConstants.DEFAULT_CHARSET));
   }
 
+
+  /**
+   * This method will copy the given file to carbon store location
+   *
+   * @param localFilePath local file name with full path
+   * @throws CarbonDataWriterException
+   */
+  public static void copyCarbonDataFileToCarbonStorePath(String localFilePath,
+      String carbonDataDirectoryPath, long fileSizeInBytes)
+      throws CarbonDataWriterException {
+    long copyStartTime = System.currentTimeMillis();
+    LOGGER.info("Copying " + localFilePath + " --> " + carbonDataDirectoryPath);
+    try {
+      CarbonFile localCarbonFile =
+          FileFactory.getCarbonFile(localFilePath, FileFactory.getFileType(localFilePath));
+      String carbonFilePath = carbonDataDirectoryPath + localFilePath
+          .substring(localFilePath.lastIndexOf(File.separator));
+      copyLocalFileToCarbonStore(carbonFilePath, localFilePath,
+          CarbonCommonConstants.BYTEBUFFER_SIZE,
+          getMaxOfBlockAndFileSize(fileSizeInBytes, localCarbonFile.getSize()));
+    } catch (IOException e) {
+      throw new CarbonDataWriterException(
+          "Problem while copying file from local store to carbon store", e);
+    }
+    LOGGER.info(
+        "Total copy time (ms) to copy file " + localFilePath + " is " + (System.currentTimeMillis()
+            - copyStartTime));
+  }
+
+  /**
+   * This method will read the local carbon data file and write to carbon data file in HDFS
+   *
+   * @param carbonStoreFilePath
+   * @param localFilePath
+   * @param bufferSize
+   * @param blockSize
+   * @throws IOException
+   */
+  private static void copyLocalFileToCarbonStore(String carbonStoreFilePath, String localFilePath,
+      int bufferSize, long blockSize) throws IOException {
+    DataOutputStream dataOutputStream = null;
+    DataInputStream dataInputStream = null;
+    try {
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("HDFS file block size for file: " + carbonStoreFilePath + " is " + blockSize
+            + " (bytes");
+      }
+      dataOutputStream = FileFactory
+          .getDataOutputStream(carbonStoreFilePath, FileFactory.getFileType(carbonStoreFilePath),
+              bufferSize, blockSize);
+      dataInputStream = FileFactory
+          .getDataInputStream(localFilePath, FileFactory.getFileType(localFilePath), bufferSize);
+      IOUtils.copyBytes(dataInputStream, dataOutputStream, bufferSize);
+    } finally {
+      CarbonUtil.closeStream(dataInputStream);
+      CarbonUtil.closeStream(dataOutputStream);
+    }
+  }
+
+  /**
+   * 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;
+    }
+    // convert to make block size more readable.
+    String readableBlockSize = ByteUtil.convertByteToReadable(blockSize);
+    String readableFileSize = ByteUtil.convertByteToReadable(fileSize);
+    String readableMaxSize = ByteUtil.convertByteToReadable(maxSize);
+    LOGGER.info(
+        "The configured block size is " + readableBlockSize + ", the actual carbon file size is "
+            + readableFileSize + ", choose the max value " + readableMaxSize
+            + " as the block size on HDFS");
+    return maxSize;
+  }
+
   /**
    * This method will be used to update the min and max values and this will be used in case of
    * old store where min and max values for measures are written opposite

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
index 2ad6327..8002e57 100644
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMap.java
@@ -28,7 +28,8 @@ import java.util.List;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datamap.dev.DataMap;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
@@ -36,7 +37,9 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
 import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
 import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.indexstore.row.DataMapRow;
 import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.executer.FilterExecuter;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
@@ -47,7 +50,7 @@ import com.google.gson.Gson;
 /**
  * Datamap implementation for min max blocklet.
  */
-public class MinMaxDataMap implements DataMap {
+public class MinMaxDataMap extends AbstractCoarseGrainDataMap {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(MinMaxDataMap.class.getName());
@@ -58,8 +61,9 @@ public class MinMaxDataMap implements DataMap {
 
   private MinMaxIndexBlockDetails[] readMinMaxDataMap;
 
-  @Override public void init(String filePath) throws MemoryException, IOException {
-    this.filePath = filePath;
+  @Override
+  public void init(DataMapModel model) throws MemoryException, IOException {
+    this.filePath = model.getFilePath();
     CarbonFile[] listFiles = getCarbonMinMaxIndexFiles(filePath, "0");
     for (int i = 0; i < listFiles.length; i++) {
       readMinMaxDataMap = readJson(listFiles[i].getPath());
@@ -76,7 +80,7 @@ public class MinMaxDataMap implements DataMap {
     });
   }
 
-  public MinMaxIndexBlockDetails[] readJson(String filePath) throws IOException {
+  private MinMaxIndexBlockDetails[] readJson(String filePath) {
     Gson gsonObjectToRead = new Gson();
     DataInputStream dataInputStream = null;
     BufferedReader buffReader = null;
@@ -90,8 +94,7 @@ public class MinMaxDataMap implements DataMap {
         return null;
       }
       dataInputStream = fileOperation.openForRead();
-      inStream = new InputStreamReader(dataInputStream,
-          CarbonCommonConstants.CARBON_DEFAULT_STREAM_ENCODEFORMAT);
+      inStream = new InputStreamReader(dataInputStream, "UTF-8");
       buffReader = new BufferedReader(inStream);
       readMinMax = gsonObjectToRead.fromJson(buffReader, MinMaxIndexBlockDetails[].class);
     } catch (IOException e) {
@@ -109,14 +112,14 @@ public class MinMaxDataMap implements DataMap {
    * @param segmentProperties
    * @return
    */
-  @Override public List<Blocklet> prune(FilterResolverIntf filterExp,
-      SegmentProperties segmentProperties) {
+  @Override
+  public List<Blocklet> prune(FilterResolverIntf filterExp,
+      SegmentProperties segmentProperties, List<String> partitions) {
     List<Blocklet> blocklets = new ArrayList<>();
 
     if (filterExp == null) {
       for (int i = 0; i < readMinMaxDataMap.length; i++) {
-        blocklets.add(new Blocklet(readMinMaxDataMap[i].getFilePath(),
-            String.valueOf(readMinMaxDataMap[i].getBlockletId())));
+        blocklets.add(new Blocklet(filePath, String.valueOf(readMinMaxDataMap[i].getBlockletId())));
       }
     } else {
       FilterExecuter filterExecuter =
@@ -126,7 +129,7 @@ public class MinMaxDataMap implements DataMap {
         BitSet bitSet = filterExecuter.isScanRequired(readMinMaxDataMap[startIndex].getMaxValues(),
             readMinMaxDataMap[startIndex].getMinValues());
         if (!bitSet.isEmpty()) {
-          blocklets.add(new Blocklet(readMinMaxDataMap[startIndex].getFilePath(),
+          blocklets.add(new Blocklet(filePath,
               String.valueOf(readMinMaxDataMap[startIndex].getBlockletId())));
         }
         startIndex++;
@@ -136,6 +139,11 @@ public class MinMaxDataMap implements DataMap {
   }
 
   @Override
+  public boolean isScanRequired(FilterResolverIntf filterExp) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
   public void clear() {
     readMinMaxDataMap = null;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e992013a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
----------------------------------------------------------------------
diff --git a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
index b196d0d..5203cb3 100644
--- a/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
+++ b/datamap/examples/src/minmaxdatamap/main/java/org/apache/carbondata/datamap/examples/MinMaxDataMapFactory.java
@@ -25,49 +25,51 @@ import java.util.List;
 
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapMeta;
-import org.apache.carbondata.core.datamap.dev.DataMap;
-import org.apache.carbondata.core.datamap.dev.DataMapFactory;
-import org.apache.carbondata.core.datamap.dev.DataMapWriter;
-import org.apache.carbondata.core.events.ChangeEvent;
-import org.apache.carbondata.core.indexstore.schema.FilterType;
+import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter;
+import org.apache.carbondata.core.datamap.dev.DataMapModel;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMap;
+import org.apache.carbondata.core.datamap.dev.cgdatamap.AbstractCoarseGrainDataMapFactory;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.events.Event;
 
 /**
  * Min Max DataMap Factory
  */
-public class MinMaxDataMapFactory implements DataMapFactory {
+public class MinMaxDataMapFactory extends AbstractCoarseGrainDataMapFactory {
 
   private AbsoluteTableIdentifier identifier;
 
-  @Override
-  public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
+  @Override public void init(AbsoluteTableIdentifier identifier, String dataMapName) {
     this.identifier = identifier;
   }
 
   /**
    * createWriter will return the MinMaxDataWriter.
+   *
    * @param segmentId
    * @return
    */
-  @Override
-  public DataMapWriter createWriter(String segmentId) {
-    return new MinMaxDataWriter();
+  @Override public AbstractDataMapWriter createWriter(String segmentId, String dataWritePath) {
+    return new MinMaxDataWriter(identifier, segmentId, dataWritePath);
   }
 
   /**
    * getDataMaps Factory method Initializes the Min Max Data Map and returns.
+   *
    * @param segmentId
    * @return
    * @throws IOException
    */
-  @Override public List<DataMap> getDataMaps(String segmentId) throws IOException {
-    List<DataMap> dataMapList = new ArrayList<>();
+  @Override public List<AbstractCoarseGrainDataMap> getDataMaps(String segmentId)
+      throws IOException {
+    List<AbstractCoarseGrainDataMap> dataMapList = new ArrayList<>();
     // Form a dataMap of Type MinMaxDataMap.
     MinMaxDataMap dataMap = new MinMaxDataMap();
     try {
-      dataMap.init(identifier.getTablePath() + "/Fact/Part0/Segment_" + segmentId + File.separator);
+      dataMap.init(new DataMapModel(
+          identifier.getTablePath() + "/Fact/Part0/Segment_" + segmentId + File.separator));
     } catch (MemoryException ex) {
 
     }
@@ -76,7 +78,6 @@ public class MinMaxDataMapFactory implements DataMapFactory {
   }
 
   /**
-   *
    * @param segmentId
    * @return
    */
@@ -86,6 +87,7 @@ public class MinMaxDataMapFactory implements DataMapFactory {
 
   /**
    * Clear the DataMap.
+   *
    * @param segmentId
    */
   @Override public void clear(String segmentId) {
@@ -94,21 +96,20 @@ public class MinMaxDataMapFactory implements DataMapFactory {
   /**
    * Clearing the data map.
    */
-  @Override
-  public void clear() {
+  @Override public void clear() {
   }
 
-  @Override public DataMap getDataMap(DataMapDistributable distributable) {
+  @Override public List<AbstractCoarseGrainDataMap> getDataMaps(DataMapDistributable distributable)
+      throws IOException {
     return null;
   }
 
-  @Override
-  public void fireEvent(ChangeEvent event) {
+  @Override public void fireEvent(Event event) {
 
   }
 
-  @Override
-  public DataMapMeta getMeta() {
-    return new DataMapMeta(new ArrayList<String>(Arrays.asList("c2")), FilterType.EQUALTO);
+  @Override public DataMapMeta getMeta() {
+    return new DataMapMeta(new ArrayList<String>(Arrays.asList("c2")),
+        new ArrayList<ExpressionType>());
   }
 }
\ No newline at end of file


[02/49] carbondata git commit: [CARBONDATA-1992] Remove partitionId in CarbonTablePath

Posted by ja...@apache.org.
[CARBONDATA-1992] Remove partitionId in CarbonTablePath

In CarbonTablePath, there is a deprecated partition id which is always 0, it should be removed to avoid confusion.

This closes #1765


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

Branch: refs/heads/carbonstore-rebase4
Commit: bf3602fc2814b596f19c8493e49d75d94c68dff5
Parents: 8d3c774
Author: Jacky Li <ja...@qq.com>
Authored: Sat Jan 6 20:28:44 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Mon Feb 26 23:55:27 2018 +0800

----------------------------------------------------------------------
 .../core/metadata/PartitionMapFileStore.java    |   0
 .../core/mutate/CarbonUpdateUtil.java           |   8 +-
 .../core/statusmanager/LoadMetadataDetails.java |   1 +
 .../SegmentUpdateStatusManager.java             |   6 +-
 .../apache/carbondata/core/util/CarbonUtil.java |   6 +-
 .../core/util/path/CarbonTablePath.java         |  55 ++++---
 .../CarbonFormatDirectoryStructureTest.java     |   4 +-
 .../hadoop/api/CarbonTableInputFormat.java      |   2 +-
 .../streaming/CarbonStreamRecordWriter.java     |   2 +-
 .../hadoop/test/util/StoreCreator.java          |   1 -
 .../presto/util/CarbonDataStoreCreator.scala    |   1 -
 .../dataload/TestLoadDataGeneral.scala          |   2 +-
 .../InsertIntoCarbonTableTestCase.scala         |   4 +-
 .../dataload/TestBatchSortDataLoad.scala        |   3 +-
 .../dataload/TestDataLoadWithFileName.scala     |   2 +-
 .../dataload/TestGlobalSortDataLoad.scala       |   4 +-
 .../testsuite/datamap/TestDataMapCommand.scala  |  34 ++--
 .../TestDataLoadingForPartitionTable.scala      |   3 +-
 .../load/DataLoadProcessBuilderOnSpark.scala    |   1 -
 .../load/DataLoadProcessorStepOnSpark.scala     |   2 +-
 .../spark/rdd/AlterTableLoadPartitionRDD.scala  | 154 +++++++++++--------
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  11 +-
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  25 ++-
 .../org/apache/spark/util/PartitionUtils.scala  |   5 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   5 +-
 .../datasources/CarbonFileFormat.scala          |   1 -
 .../partition/TestAlterPartitionTable.scala     |   2 +-
 .../bucketing/TableBucketingTestCase.scala      |   2 +
 .../loading/CarbonDataLoadConfiguration.java    |  10 --
 .../loading/DataLoadProcessBuilder.java         |   1 -
 .../loading/TableProcessingOperations.java      |   3 +-
 .../sort/impl/ParallelReadMergeSorterImpl.java  |   4 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |  15 +-
 .../UnsafeBatchParallelReadMergeSorterImpl.java |   7 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |  21 ++-
 .../CarbonRowDataWriterProcessorStepImpl.java   |  33 ++--
 .../steps/DataWriterBatchProcessorStepImpl.java |  25 +--
 .../steps/DataWriterProcessorStepImpl.java      |  22 +--
 .../processing/merger/CarbonDataMergerUtil.java |   6 +-
 .../merger/CompactionResultSortProcessor.java   |   4 +-
 .../sort/sortdata/SortParameters.java           |  16 +-
 .../store/CarbonFactDataHandlerModel.java       |   3 +-
 .../util/CarbonDataProcessorUtil.java           |   9 +-
 .../processing/util/CarbonLoaderUtil.java       |  12 +-
 .../processing/util/DeleteLoadFolders.java      |   7 +-
 .../carbondata/processing/StoreCreator.java     |   1 -
 .../carbondata/streaming/StreamHandoffRDD.scala |   1 -
 .../streaming/StreamSinkFactory.scala           |   2 +-
 .../streaming/CarbonAppendableStreamSink.scala  |   8 +-
 49 files changed, 274 insertions(+), 282 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/PartitionMapFileStore.java
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
index de98fa8..18eae11 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
@@ -319,9 +319,7 @@ public class CarbonUpdateUtil {
     CarbonTablePath carbonTablePath = CarbonStorePath
             .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
                     absoluteTableIdentifier.getCarbonTableIdentifier());
-    // as of now considering only partition 0.
-    String partitionId = "0";
-    String partitionDir = carbonTablePath.getPartitionDir(partitionId);
+    String partitionDir = carbonTablePath.getPartitionDir();
     CarbonFile file =
             FileFactory.getCarbonFile(partitionDir, FileFactory.getFileType(partitionDir));
     if (!file.exists()) {
@@ -402,7 +400,7 @@ public class CarbonUpdateUtil {
   }
 
   public static long getLatestTaskIdForSegment(String segmentId, CarbonTablePath tablePath) {
-    String segmentDirPath = tablePath.getCarbonDataDirectoryPath("0", segmentId);
+    String segmentDirPath = tablePath.getCarbonDataDirectoryPath(segmentId);
 
     // scan all the carbondata files and get the latest task ID.
     CarbonFile segment =
@@ -467,7 +465,7 @@ public class CarbonUpdateUtil {
               || segment.getSegmentStatus() == SegmentStatus.LOAD_PARTIAL_SUCCESS) {
 
         // take the list of files from this segment.
-        String segmentPath = carbonTablePath.getCarbonDataDirectoryPath("0", segment.getLoadName());
+        String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segment.getLoadName());
         CarbonFile segDir =
                 FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
         CarbonFile[] allSegmentFiles = segDir.listFiles();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
index a0fa67d..b6a9e36 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
@@ -132,6 +132,7 @@ public class LoadMetadataDetails implements Serializable {
     return partitionCount;
   }
 
+  @Deprecated
   public void setPartitionCount(String partitionCount) {
     this.partitionCount = partitionCount;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
index 71b6ba8..2edb379 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
@@ -202,7 +202,7 @@ public class SegmentUpdateStatusManager {
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     String endTimeStamp = "";
     String startTimeStamp = "";
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
+    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
     for (LoadMetadataDetails eachSeg : segmentDetails) {
@@ -437,7 +437,7 @@ public class SegmentUpdateStatusManager {
         .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
 
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath("0", segmentId.getSegmentNo());
+    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId.getSegmentNo());
 
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
@@ -880,7 +880,7 @@ public class SegmentUpdateStatusManager {
 
     // filter out the fact files.
 
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
+    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index eb0a9d7..52305bd 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -1105,7 +1105,7 @@ public final class CarbonUtil {
     // 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(),
+        .getCarbonIndexFilePath(taskId, tableBlockInfoList.get(0).getSegmentId(),
             bucketNumber, CarbonTablePath.DataFileUtil
                 .getTimeStampFromFileName(tableBlockInfoList.get(0).getFilePath()),
             tableBlockInfoList.get(0).getVersion());
@@ -1348,7 +1348,7 @@ public final class CarbonUtil {
     // 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(),
+        .getCarbonIndexFilePath(taskId, tableBlockInfoList.get(0).getSegmentId(),
             bucketNumber, CarbonTablePath.DataFileUtil
                 .getTimeStampFromFileName(tableBlockInfoList.get(0).getFilePath()),
             tableBlockInfoList.get(0).getVersion());
@@ -2305,7 +2305,7 @@ public final class CarbonUtil {
     long carbonDataSize = 0L;
     long carbonIndexSize = 0L;
     HashMap<String, Long> dataAndIndexSize = new HashMap<String, Long>();
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
+    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
     FileFactory.FileType fileType = FileFactory.getFileType(segmentPath);
     switch (fileType) {
       case HDFS:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index d70d9ef..293257b 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -267,15 +267,14 @@ public class CarbonTablePath extends Path {
   /**
    * Gets absolute path of data file
    *
-   * @param partitionId         unique partition identifier
    * @param segmentId           unique partition identifier
    * @param filePartNo          data file part number
    * @param factUpdateTimeStamp unique identifier to identify an update
    * @return absolute path of data file stored in carbon data format
    */
-  public String getCarbonDataFilePath(String partitionId, String segmentId, Integer filePartNo,
-      Long taskNo, int batchNo, int bucketNumber, String factUpdateTimeStamp) {
-    return getSegmentDir(partitionId, segmentId) + File.separator + getCarbonDataFileName(
+  public String getCarbonDataFilePath(String segmentId, Integer filePartNo, Long taskNo,
+      int batchNo, int bucketNumber, String factUpdateTimeStamp) {
+    return getSegmentDir(segmentId) + File.separator + getCarbonDataFileName(
         filePartNo, taskNo, bucketNumber, batchNo, factUpdateTimeStamp);
   }
 
@@ -284,13 +283,12 @@ public class CarbonTablePath extends Path {
    * 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, final String bucketNumber) {
-    String segmentDir = getSegmentDir(partitionId, segmentId);
+  public String getCarbonIndexFilePath(final String taskId, final String segmentId,
+      final String bucketNumber) {
+    String segmentDir = getSegmentDir(segmentId);
     CarbonFile carbonFile =
         FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir));
 
@@ -306,9 +304,8 @@ public class CarbonTablePath extends Path {
     if (files.length > 0) {
       return files[0].getAbsolutePath();
     } else {
-      throw new RuntimeException("Missing Carbon index file for partition["
-          + partitionId + "] Segment[" + segmentId + "], taskId[" + taskId
-          + "]");
+      throw new RuntimeException("Missing Carbon index file for Segment[" + segmentId + "], "
+          + "taskId[" + taskId + "]");
     }
   }
 
@@ -316,8 +313,6 @@ public class CarbonTablePath extends Path {
    * Below method will be used to get the carbon index file path
    * @param taskId
    *        task id
-   * @param partitionId
-   *        partition id
    * @param segmentId
    *        segment id
    * @param bucketNumber
@@ -326,28 +321,27 @@ public class CarbonTablePath extends Path {
    *        timestamp
    * @return carbon index file path
    */
-  public String getCarbonIndexFilePath(String taskId, String partitionId, String segmentId,
-      String bucketNumber, String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
+  public String getCarbonIndexFilePath(String taskId, String segmentId, String bucketNumber,
+      String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
     switch (columnarFormatVersion) {
       case V1:
       case V2:
-        return getCarbonIndexFilePath(taskId, partitionId, segmentId, bucketNumber);
+        return getCarbonIndexFilePath(taskId, segmentId, bucketNumber);
       default:
-        String segmentDir = getSegmentDir(partitionId, segmentId);
+        String segmentDir = getSegmentDir(segmentId);
         return segmentDir + File.separator + getCarbonIndexFileName(taskId,
             Integer.parseInt(bucketNumber), timeStamp);
     }
   }
 
-  public String getCarbonIndexFilePath(String taskId, String partitionId, String segmentId,
-      int batchNo, String bucketNumber, String timeStamp,
-      ColumnarFormatVersion columnarFormatVersion) {
+  public String getCarbonIndexFilePath(String taskId, String segmentId, int batchNo,
+      String bucketNumber, String timeStamp, ColumnarFormatVersion columnarFormatVersion) {
     switch (columnarFormatVersion) {
       case V1:
       case V2:
-        return getCarbonIndexFilePath(taskId, partitionId, segmentId, bucketNumber);
+        return getCarbonIndexFilePath(taskId, segmentId, bucketNumber);
       default:
-        String segmentDir = getSegmentDir(partitionId, segmentId);
+        String segmentDir = getSegmentDir(segmentId);
         return segmentDir + File.separator + getCarbonIndexFileName(Long.parseLong(taskId),
             Integer.parseInt(bucketNumber), batchNo, timeStamp);
     }
@@ -364,12 +358,11 @@ public class CarbonTablePath extends Path {
   /**
    * Gets absolute path of data file
    *
-   * @param partitionId unique partition identifier
    * @param segmentId   unique partition identifier
    * @return absolute path of data file stored in carbon data format
    */
-  public String getCarbonDataDirectoryPath(String partitionId, String segmentId) {
-    return getSegmentDir(partitionId, segmentId);
+  public String getCarbonDataDirectoryPath(String segmentId) {
+    return getSegmentDir(segmentId);
   }
 
   /**
@@ -407,12 +400,16 @@ public class CarbonTablePath extends Path {
     return segmentDir + File.separator + getCarbonStreamIndexFileName();
   }
 
-  public String getSegmentDir(String partitionId, String segmentId) {
-    return getPartitionDir(partitionId) + File.separator + SEGMENT_PREFIX + segmentId;
+  public String getSegmentDir(String segmentId) {
+    return getPartitionDir() + File.separator + SEGMENT_PREFIX + segmentId;
   }
 
-  public String getPartitionDir(String partitionId) {
-    return getFactDir() + File.separator + PARTITION_PREFIX + partitionId;
+  // This partition is not used in any code logic, just keep backward compatibility
+  public static final String DEPRECATED_PATITION_ID = "0";
+
+  public String getPartitionDir() {
+    return getFactDir() + File.separator + PARTITION_PREFIX +
+        CarbonTablePath.DEPRECATED_PATITION_ID;
   }
 
   private String getMetaDataDir() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
index 5549806..a1ccab3 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/path/CarbonFormatDirectoryStructureTest.java
@@ -53,8 +53,8 @@ public class CarbonFormatDirectoryStructureTest {
         .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.dictmeta"));
     assertTrue(carbonTablePath.getSortIndexFilePath("t1_c1").replace("\\", "/")
         .equals(CARBON_STORE + "/d1/t1/Metadata/t1_c1.sortindex"));
-    assertTrue(carbonTablePath.getCarbonDataFilePath("1", "2", 3, 4L,  0, 0, "999").replace("\\", "/")
-        .equals(CARBON_STORE + "/d1/t1/Fact/Part1/Segment_2/part-3-4_batchno0-0-999.carbondata"));
+    assertTrue(carbonTablePath.getCarbonDataFilePath("2", 3, 4L,  0, 0, "999").replace("\\", "/")
+        .equals(CARBON_STORE + "/d1/t1/Fact/Part0/Segment_2/part-3-4_batchno0-0-999.carbondata"));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 96b0b21..069e1f7 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -499,7 +499,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
       long minSize = Math.max(getFormatMinSplitSize(), getMinSplitSize(job));
       long maxSize = getMaxSplitSize(job);
       for (Segment segment : streamSegments) {
-        String segmentDir = tablePath.getSegmentDir("0", segment.getSegmentNo());
+        String segmentDir = tablePath.getSegmentDir(segment.getSegmentNo());
         FileFactory.FileType fileType = FileFactory.getFileType(segmentDir);
         if (FileFactory.isFileExist(segmentDir, fileType)) {
           String indexName = CarbonTablePath.getCarbonStreamIndexFileName();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
index 364a6a6..3ef8afc 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordWriter.java
@@ -129,7 +129,7 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
 
     CarbonTablePath tablePath =
         CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
-    segmentDir = tablePath.getSegmentDir("0", segmentId);
+    segmentDir = tablePath.getSegmentDir(segmentId);
     fileName = CarbonTablePath.getCarbonDataFileName(0, taskNo, 0, 0, "0");
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
index fbf33d6..ac17c4e 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/test/util/StoreCreator.java
@@ -154,7 +154,6 @@ public class StoreCreator {
     loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(","));
     loadModel.setTaskNo("0");
     loadModel.setSegmentId("0");
-    loadModel.setPartitionId("0");
     loadModel.setFactTimeStamp(System.currentTimeMillis());
     loadModel.setMaxColumns("10");
     return loadModel;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index 7b5c311..a41e738 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -135,7 +135,6 @@ object CarbonDataStoreCreator {
       loadModel.setCsvHeaderColumns(loadModel.getCsvHeader.split(","))
       loadModel.setTaskNo("0")
       loadModel.setSegmentId("0")
-      loadModel.setPartitionId("0")
       loadModel.setFactTimeStamp(System.currentTimeMillis())
       loadModel.setMaxColumns("15")
       executeGraph(loadModel, storePath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index 09ca9e5..c84ae6b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -49,7 +49,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterEach {
       tableName: String): Boolean = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(datbaseName, tableName)
     val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir("0")
+      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
     val fileType: FileFactory.FileType = FileFactory.getFileType(partitionPath)
     val carbonFile = FileFactory.getCarbonFile(partitionPath, fileType)
     val segments: ArrayBuffer[String] = ArrayBuffer()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
index d59f0b5..5cc4156 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/InsertIntoCarbonTableTestCase.scala
@@ -232,7 +232,7 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     checkAnswer(sql("select count(*) from CarbonOverwrite"), sql("select count(*) from HiveOverwrite"))
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "carbonoverwrite")
     val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir("0")
+      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
     val folder = new File(partitionPath)
     assert(folder.isDirectory)
     assert(folder.list().length == 1)
@@ -255,7 +255,7 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     checkAnswer(sql("select count(*) from TCarbonSourceOverwrite"), sql("select count(*) from HiveOverwrite"))
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "tcarbonsourceoverwrite")
     val partitionPath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir("0")
+      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier).getPartitionDir
     val folder = new File(partitionPath)
 
     assert(folder.isDirectory)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
index 4af9d54..42ac4df 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestBatchSortDataLoad.scala
@@ -193,9 +193,8 @@ class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
       CarbonCommonConstants.DATABASE_DEFAULT_NAME,
       tableName
     )
-    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
     val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0", segmentNo)
+    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath(segmentNo)
     new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
index dae0962..db0a62c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithFileName.scala
@@ -49,7 +49,7 @@ class TestDataLoadWithFileName extends QueryTest with BeforeAndAfterAll {
     val indexReader = new CarbonIndexFileReader()
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "test_table_v3")
     val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0", "0")
+    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0")
     val carbonIndexPaths = new File(segmentDir)
       .listFiles(new FilenameFilter {
         override def accept(dir: File, name: String): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index 0d9e0fd..479db50 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -272,7 +272,7 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
     sql(s"LOAD DATA LOCAL INPATH '$inputPath' INTO TABLE carbon_globalsort")
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "carbon_globalsort")
     val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getSegmentDir("0", "0")
+    val segmentDir = carbonTablePath.getSegmentDir("0")
     assertResult(Math.max(7, defaultParallelism) + 1)(new File(segmentDir).listFiles().length)
   }
 
@@ -379,7 +379,7 @@ class TestGlobalSortDataLoad extends QueryTest with BeforeAndAfterEach with Befo
   private def getIndexFileCount(tableName: String, segmentNo: String = "0"): Int = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", tableName)
     val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath("0", segmentNo)
+    val segmentDir = carbonTablePath.getCarbonDataDirectoryPath(segmentNo)
     new SegmentIndexFileStore().getIndexFilesFromSegment(segmentDir).size()
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
index f403b3e..d60b7db 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
@@ -220,19 +220,27 @@ class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
   }
 
   test("test if preaggregate load is successfull for hivemetastore") {
-    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE, "true")
-    sql("DROP TABLE IF EXISTS maintable")
-    sql(
-      """
-        | CREATE TABLE maintable(id int, name string, city string, age int)
-        | STORED BY 'org.apache.carbondata.format'
-      """.stripMargin)
-    sql(
-      s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id,sum(age) from maintable group by id"""
-        .stripMargin)
-    sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
-    checkAnswer(sql(s"select * from maintable_preagg_sum"),
-      Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE, "true")
+      sql("DROP TABLE IF EXISTS maintable")
+      sql(
+        """
+          | CREATE TABLE maintable(id int, name string, city string, age int)
+          | STORED BY 'org.apache.carbondata.format'
+        """.stripMargin)
+      sql(
+        s"""create datamap preagg_sum on table maintable using 'preaggregate' as select id,sum(age) from maintable group by id"""
+
+          .stripMargin)
+      sql(s"LOAD DATA LOCAL INPATH '$testData' into table maintable")
+      checkAnswer(sql(s"select * from maintable_preagg_sum"),
+        Seq(Row(1, 31), Row(2, 27), Row(3, 70), Row(4, 55)))
+    } finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE,
+          CarbonCommonConstants.ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT)
+    }
   }
 
   test("test preaggregate load for decimal column for hivemetastore") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
index ed151bd..0a21aed 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDataLoadingForPartitionTable.scala
@@ -63,7 +63,7 @@ class TestDataLoadingForPartitionTable extends QueryTest with BeforeAndAfterAll
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
     val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
       carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath("0", segmentId)
+    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {
@@ -87,6 +87,7 @@ class TestDataLoadingForPartitionTable extends QueryTest with BeforeAndAfterAll
         |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
         |  utilization int,salary int)
         | PARTITIONED BY (empno int)
+        |
         | STORED BY 'org.apache.carbondata.format'
         | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
       """.stripMargin)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
index 30e4fc9..e1bd84b 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessBuilderOnSpark.scala
@@ -56,7 +56,6 @@ object DataLoadProcessBuilderOnSpark {
         .map(DataLoadProcessorStepOnSpark.toStringArrayRow(_, columnCount))
     }
 
-    model.setPartitionId("0")
     val sc = sparkSession.sparkContext
     val modelBroadcast = sc.broadcast(model)
     val partialSuccessAccum = sc.accumulator(0, "Partial Success Accumulator")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
index 4b7d3f7..5124247 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
@@ -248,7 +248,7 @@ object DataLoadProcessorStepOnSpark {
 
       dataWriter = new DataWriterProcessorStepImpl(conf)
 
-      val dataHandlerModel = dataWriter.getDataHandlerModel(0)
+      val dataHandlerModel = dataWriter.getDataHandlerModel
       var dataHandler: CarbonFactHandler = null
       var rowsNotExist = true
       while (rows.hasNext) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
index 76c99f2..9de8dc9 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
@@ -18,19 +18,21 @@
 package org.apache.carbondata.spark.rdd
 
 import scala.collection.JavaConverters._
+import scala.util.Random
 
-import org.apache.spark.{Partition, TaskContext}
+import org.apache.spark.{Partition, SparkEnv, TaskContext}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.execution.command.AlterPartitionModel
 import org.apache.spark.util.PartitionUtils
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.processing.loading.TableProcessingOperations
 import org.apache.carbondata.processing.partition.spliter.RowResultProcessor
 import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.AlterPartitionResult
-import org.apache.carbondata.spark.util.CommonUtil
+import org.apache.carbondata.spark.util.{CommonUtil, Util}
 
 class AlterTableLoadPartitionRDD[K, V](alterPartitionModel: AlterPartitionModel,
     result: AlterPartitionResult[K, V],
@@ -39,76 +41,96 @@ class AlterTableLoadPartitionRDD[K, V](alterPartitionModel: AlterPartitionModel,
     identifier: AbsoluteTableIdentifier,
     prev: RDD[Array[AnyRef]]) extends RDD[(K, V)](prev) {
 
-    var storeLocation: String = null
-    val carbonLoadModel = alterPartitionModel.carbonLoadModel
-    val segmentId = alterPartitionModel.segmentId
-    val oldPartitionIds = alterPartitionModel.oldPartitionIds
-    val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-    val databaseName = carbonTable.getDatabaseName
-    val factTableName = carbonTable.getTableName
-    val partitionInfo = carbonTable.getPartitionInfo(factTableName)
+  var storeLocation: String = null
+  val carbonLoadModel = alterPartitionModel.carbonLoadModel
+  val segmentId = alterPartitionModel.segmentId
+  val oldPartitionIds = alterPartitionModel.oldPartitionIds
+  val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+  val databaseName = carbonTable.getDatabaseName
+  val factTableName = carbonTable.getTableName
+  val partitionInfo = carbonTable.getPartitionInfo(factTableName)
 
-    override protected def getPartitions: Array[Partition] = {
-        val sc = alterPartitionModel.sqlContext.sparkContext
-        sc.setLocalProperty("spark.scheduler.pool", "DDL")
-        sc.setLocalProperty("spark.job.interruptOnCancel", "true")
-        firstParent[Array[AnyRef]].partitions
-    }
-
-    override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
-        val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-        val rows = firstParent[Array[AnyRef]].iterator(split, context).toList.asJava
-        val iter = new Iterator[(K, V)] {
-            val partitionId = partitionInfo.getPartitionId(split.index)
-            carbonLoadModel.setTaskNo(String.valueOf(partitionId))
-            carbonLoadModel.setSegmentId(segmentId)
-            carbonLoadModel.setPartitionId("0")
-            CommonUtil.setTempStoreLocation(split.index, carbonLoadModel, false, true)
+  override protected def getPartitions: Array[Partition] = {
+    val sc = alterPartitionModel.sqlContext.sparkContext
+    sc.setLocalProperty("spark.scheduler.pool", "DDL")
+    sc.setLocalProperty("spark.job.interruptOnCancel", "true")
+    firstParent[Array[AnyRef]].partitions
+  }
 
-            val tempStoreLoc = CarbonDataProcessorUtil.getLocalDataFolderLocation(databaseName,
-                factTableName,
-                carbonLoadModel.getTaskNo,
-                "0",
-                segmentId,
-                false,
-                true
-            )
+  override def compute(split: Partition, context: TaskContext): Iterator[(K, V)] = {
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+    val rows = firstParent[Array[AnyRef]].iterator(split, context).toList.asJava
+    val iter = new Iterator[(K, V)] {
+      val partitionId = partitionInfo.getPartitionId(split.index)
+      carbonLoadModel.setTaskNo(String.valueOf(partitionId))
+      carbonLoadModel.setSegmentId(segmentId)
+      CommonUtil.setTempStoreLocation(split.index, carbonLoadModel, false, true)
+      val tempLocationKey = CarbonDataProcessorUtil
+        .getTempStoreLocationKey(carbonLoadModel.getDatabaseName,
+          carbonLoadModel.getTableName,
+          segmentId,
+          carbonLoadModel.getTaskNo,
+          false,
+          true)
+      // this property is used to determine whether temp location for carbon is inside
+      // container temp dir or is yarn application directory.
+      val carbonUseLocalDir = CarbonProperties.getInstance()
+        .getProperty("carbon.use.local.dir", "false")
 
-            val loadStatus = if (rows.isEmpty) {
-                LOGGER.info("After repartition this split, NO target rows to write back.")
-                true
-            } else {
-                val segmentProperties = PartitionUtils.getSegmentProperties(identifier,
-                    segmentId, partitionIds.toList, oldPartitionIds, partitionInfo, carbonTable)
-                val processor = new RowResultProcessor(
-                    carbonTable,
-                    carbonLoadModel,
-                    segmentProperties,
-                    tempStoreLoc,
-                    bucketId)
-                try {
-                    processor.execute(rows)
-                } catch {
-                    case e: Exception =>
-                        sys.error(s"Exception when executing Row result processor ${e.getMessage}")
-                } finally {
-                    TableProcessingOperations
-                      .deleteLocalDataLoadFolderLocation(carbonLoadModel, false, true)
-                }
-            }
+      if (carbonUseLocalDir.equalsIgnoreCase("true")) {
 
-            val loadResult = segmentId
-            var finished = false
+        val storeLocations = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
+        if (null != storeLocations && storeLocations.nonEmpty) {
+          storeLocation = storeLocations(Random.nextInt(storeLocations.length))
+        }
+        if (storeLocation == null) {
+          storeLocation = System.getProperty("java.io.tmpdir")
+        }
+      } else {
+        storeLocation = System.getProperty("java.io.tmpdir")
+      }
+      storeLocation = storeLocation + '/' + System.nanoTime() + '/' + split.index
+      CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation)
+      LOGGER.info(s"Temp storeLocation taken is $storeLocation")
 
-            override def hasNext: Boolean = {
-                !finished
-            }
+      val tempStoreLoc = CarbonDataProcessorUtil.getLocalDataFolderLocation(
+        databaseName, factTableName, carbonLoadModel.getTaskNo, segmentId, false, true)
 
-            override def next(): (K, V) = {
-                finished = true
-                result.getKey(loadResult, loadStatus)
-            }
+      val loadStatus = if (rows.isEmpty) {
+        LOGGER.info("After repartition this split, NO target rows to write back.")
+        true
+      } else {
+        val segmentProperties = PartitionUtils.getSegmentProperties(identifier,
+          segmentId, partitionIds.toList, oldPartitionIds, partitionInfo, carbonTable)
+        val processor = new RowResultProcessor(
+          carbonTable,
+          carbonLoadModel,
+          segmentProperties,
+          tempStoreLoc,
+          bucketId)
+        try {
+          processor.execute(rows)
+        } catch {
+          case e: Exception =>
+            sys.error(s"Exception when executing Row result processor ${ e.getMessage }")
+        } finally {
+          TableProcessingOperations
+            .deleteLocalDataLoadFolderLocation(carbonLoadModel, false, true)
         }
-        iter
+      }
+
+      val loadResult = segmentId
+      var finished = false
+
+      override def hasNext: Boolean = {
+        !finished
+      }
+
+      override def next(): (K, V) = {
+        finished = true
+        result.getKey(loadResult, loadStatus)
+      }
     }
+    iter
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index e0dcffd..ab3ab5d 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -189,16 +189,9 @@ class CarbonMergerRDD[K, V](
             }
         }
 
-        val tempStoreLoc = CarbonDataProcessorUtil.getLocalDataFolderLocation(databaseName,
-          factTableName,
-          carbonLoadModel.getTaskNo,
-          "0",
-          mergeNumber,
-          true,
-          false
-        )
+        val tempStoreLoc = CarbonDataProcessorUtil.getLocalDataFolderLocation(
+          databaseName, factTableName, carbonLoadModel.getTaskNo, mergeNumber, true, false)
 
-        carbonLoadModel.setPartitionId("0")
         var processor: AbstractResultProcessor = null
         if (restructuredBlockExists) {
           LOGGER.info("CompactionResultSortProcessor flow is selected")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 72d0484..1fa1689 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -43,6 +43,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, ThreadLocalTaskInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.processing.loading.{DataLoadExecutor, FailureCauses, TableProcessingOperations}
 import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInputFormat, CSVRecordReaderIterator}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
@@ -129,7 +130,8 @@ class SparkPartitionLoader(model: CarbonLoadModel,
       System.setProperty("carbon.properties.filepath",
         System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties")
     }
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance.initPartitonInfo(model.getPartitionId)
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance.initPartitonInfo(
+      CarbonTablePath.DEPRECATED_PATITION_ID)
     CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true")
     CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1")
     CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true")
@@ -219,14 +221,13 @@ class NewCarbonDataLoadRDD[K, V](
   override def internalCompute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
     val iter = new Iterator[(K, V)] {
-      var partitionID = "0"
       val loadMetadataDetails = new LoadMetadataDetails()
       val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
       var model: CarbonLoadModel = _
       val uniqueLoadStatusId =
         carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + theSplit.index
       try {
-        loadMetadataDetails.setPartitionCount(partitionID)
+        loadMetadataDetails.setPartitionCount(CarbonTablePath.DEPRECATED_PATITION_ID)
         loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS)
 
         val preFetch = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
@@ -264,7 +265,7 @@ class NewCarbonDataLoadRDD[K, V](
         // So print the data load statistics only in case of non failure case
         if (SegmentStatus.LOAD_FAILURE != loadMetadataDetails.getSegmentStatus) {
           CarbonTimeStatisticsFactory.getLoadStatisticsInstance
-            .printStatisticsInfo(model.getPartitionId)
+            .printStatisticsInfo(CarbonTablePath.DEPRECATED_PATITION_ID)
         }
       }
 
@@ -287,8 +288,8 @@ class NewCarbonDataLoadRDD[K, V](
         val fileList: java.util.List[String] = new java.util.ArrayList[String](
             CarbonCommonConstants.CONSTANT_SIZE_TEN)
         CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, fileList, ",")
-        model = carbonLoadModel.getCopyWithPartition(partitionID, fileList,
-            carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
+        model = carbonLoadModel.getCopyWithPartition(
+          carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
         StandardLogService.setThreadName(StandardLogService
           .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)
           , ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId + "")
@@ -351,7 +352,6 @@ class NewDataFrameLoaderRDD[K, V](
 
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
     val iter = new Iterator[(K, V)] {
-      val partitionID = "0"
       val loadMetadataDetails = new LoadMetadataDetails()
       val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
       val model: CarbonLoadModel = carbonLoadModel
@@ -359,9 +359,8 @@ class NewDataFrameLoaderRDD[K, V](
         carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + theSplit.index
       try {
 
-        loadMetadataDetails.setPartitionCount(partitionID)
+        loadMetadataDetails.setPartitionCount(CarbonTablePath.DEPRECATED_PATITION_ID)
         loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS)
-        carbonLoadModel.setPartitionId(partitionID)
         carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
         carbonLoadModel.setPreFetch(false)
 
@@ -406,7 +405,7 @@ class NewDataFrameLoaderRDD[K, V](
         // So print the data load statistics only in case of non failure case
         if (SegmentStatus.LOAD_FAILURE != loadMetadataDetails.getSegmentStatus) {
           CarbonTimeStatisticsFactory.getLoadStatisticsInstance
-            .printStatisticsInfo(model.getPartitionId)
+            .printStatisticsInfo(CarbonTablePath.DEPRECATED_PATITION_ID)
         }
       }
       var finished = false
@@ -542,7 +541,6 @@ class PartitionTableDataLoaderRDD[K, V](
   override def internalCompute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
     val iter = new Iterator[(K, V)] {
-      val partitionID = "0"
       val loadMetadataDetails = new LoadMetadataDetails()
       val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
       val model: CarbonLoadModel = carbonLoadModel
@@ -552,9 +550,8 @@ class PartitionTableDataLoaderRDD[K, V](
         carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + theSplit.index
       try {
 
-        loadMetadataDetails.setPartitionCount(partitionID)
+        loadMetadataDetails.setPartitionCount(CarbonTablePath.DEPRECATED_PATITION_ID)
         loadMetadataDetails.setSegmentStatus(SegmentStatus.SUCCESS)
-        carbonLoadModel.setPartitionId(partitionID)
         carbonLoadModel.setTaskNo(String.valueOf(partitionInfo.getPartitionId(theSplit.index)))
         carbonLoadModel.setPreFetch(false)
         val recordReaders = Array[CarbonIterator[Array[AnyRef]]] {
@@ -590,7 +587,7 @@ class PartitionTableDataLoaderRDD[K, V](
         // So print the data load statistics only in case of non failure case
         if (SegmentStatus.LOAD_FAILURE != loadMetadataDetails.getSegmentStatus) {
           CarbonTimeStatisticsFactory.getLoadStatisticsInstance
-            .printStatisticsInfo(model.getPartitionId)
+            .printStatisticsInfo(CarbonTablePath.DEPRECATED_PATITION_ID)
         }
       }
       var finished = false

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
index 0498b25..3c871db 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/PartitionUtils.scala
@@ -190,8 +190,9 @@ object PartitionUtils {
         val batchNo = CarbonTablePath.DataFileUtil.getBatchNoFromTaskNo(taskNo)
         val taskId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(taskNo)
         val bucketNumber = CarbonTablePath.DataFileUtil.getBucketNo(path)
-        val indexFilePath = carbonTablePath.getCarbonIndexFilePath(String.valueOf(taskId), "0",
-          segmentId, batchNo, String.valueOf(bucketNumber), timestamp, version)
+        val indexFilePath = carbonTablePath.getCarbonIndexFilePath(
+          String.valueOf(taskId), segmentId, batchNo, String.valueOf(bucketNumber),
+          timestamp, version)
         // indexFilePath could be duplicated when multiple data file related to one index file
         if (indexFilePath != null && !pathList.contains(indexFilePath)) {
           pathList.add(indexFilePath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 1695a13..09484c4 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -594,7 +594,6 @@ object CarbonDataRDDFactory {
 
         override def getPartition(key: Any): Int = {
           val segId = key.asInstanceOf[String]
-          // partitionId
           segmentIdIndex(segId) * parallelism + Random.nextInt(parallelism)
         }
       }
@@ -628,7 +627,6 @@ object CarbonDataRDDFactory {
     val rddResult = new updateResultImpl()
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
     val resultIter = new Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] {
-      val partitionID = "0"
       val loadMetadataDetails = new LoadMetadataDetails
       val executionErrors = ExecutionErrors(FailureCauses.NONE, "")
       var uniqueLoadStatusId = ""
@@ -639,10 +637,9 @@ object CarbonDataRDDFactory {
                              CarbonCommonConstants.UNDERSCORE +
                              (index + "_0")
 
-        loadMetadataDetails.setPartitionCount(partitionID)
+        loadMetadataDetails.setPartitionCount(CarbonTablePath.DEPRECATED_PATITION_ID)
         loadMetadataDetails.setLoadName(segId)
         loadMetadataDetails.setSegmentStatus(SegmentStatus.LOAD_FAILURE)
-        carbonLoadModel.setPartitionId(partitionID)
         carbonLoadModel.setSegmentId(segId)
         carbonLoadModel.setTaskNo(String.valueOf(index))
         carbonLoadModel.setFactTimeStamp(updateModel.get.updatedTimeStamp)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
index bff65be..b4d3bea 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/datasources/CarbonFileFormat.scala
@@ -110,7 +110,6 @@ with Serializable {
       model,
       conf
     )
-    model.setPartitionId("0")
     model.setUseOnePass(options.getOrElse("onepass", "false").toBoolean)
     model.setDictionaryServerHost(options.getOrElse("dicthost", null))
     model.setDictionaryServerPort(options.getOrElse("dictport", "-1").toInt)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
index b5325ef..aadee81 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
@@ -858,7 +858,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
   def getDataFiles(carbonTable: CarbonTable, segmentId: String): Array[CarbonFile] = {
     val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
       carbonTable.getTablePath)
-    val segmentDir = tablePath.getCarbonDataDirectoryPath("0", segmentId)
+    val segmentDir = tablePath.getCarbonDataDirectoryPath(segmentId)
     val carbonFile = FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir))
     val dataFiles = carbonFile.listFiles(new CarbonFileFilter() {
       override def accept(file: CarbonFile): Boolean = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
index 102df39..9da7244 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
@@ -173,6 +173,7 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       case s: ShuffleExchange => shuffleExists = true
     }
     assert(!shuffleExists, "shuffle should not exist on bucket tables")
+    sql("DROP TABLE bucketed_parquet_table")
   }
 
   test("test create table with bucket join of carbon table and non bucket parquet table") {
@@ -197,6 +198,7 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
       case s: ShuffleExchange => shuffleExists = true
     }
     assert(shuffleExists, "shuffle should exist on non bucket tables")
+    sql("DROP TABLE parquet_table")
   }
 
   test("test scalar subquery with equal") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
index b7270b9..895fb79 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
@@ -40,8 +40,6 @@ public class CarbonDataLoadConfiguration {
 
   private String[] header;
 
-  private String partitionId;
-
   private String segmentId;
 
   private String taskNo;
@@ -194,14 +192,6 @@ public class CarbonDataLoadConfiguration {
     this.tableIdentifier = tableIdentifier;
   }
 
-  public String getPartitionId() {
-    return partitionId;
-  }
-
-  public void setPartitionId(String partitionId) {
-    this.partitionId = partitionId;
-  }
-
   public String getSegmentId() {
     return segmentId;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
index f5b29e7..fc2796a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
@@ -182,7 +182,6 @@ public final class DataLoadProcessBuilder {
     configuration.setTableIdentifier(identifier);
     configuration.setSchemaUpdatedTimeStamp(carbonTable.getTableLastUpdatedTime());
     configuration.setHeader(loadModel.getCsvHeaderColumns());
-    configuration.setPartitionId(loadModel.getPartitionId());
     configuration.setSegmentId(loadModel.getSegmentId());
     configuration.setTaskNo(loadModel.getTaskNo());
     configuration.setDataLoadProperty(DataLoadProcessorConstants.COMPLEX_DELIMITERS,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java b/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
index e2be79c..a8db6c9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/TableProcessingOperations.java
@@ -63,8 +63,7 @@ public class TableProcessingOperations {
 
     //delete folder which metadata no exist in tablestatus
     for (int i = 0; i < carbonTable.getPartitionCount(); i++) {
-      final String partitionCount = i + "";
-      String partitionPath = carbonTablePath.getPartitionDir(partitionCount);
+      String partitionPath = carbonTablePath.getPartitionDir();
       FileFactory.FileType fileType = FileFactory.getFileType(partitionPath);
       if (FileFactory.isFileExist(partitionPath, fileType)) {
         CarbonFile carbonFile = FileFactory.getCarbonFile(partitionPath, fileType);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterImpl.java
index 6432d38..fcc88b5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterImpl.java
@@ -73,8 +73,8 @@ public class ParallelReadMergeSorterImpl extends AbstractMergeSorter {
     String[] storeLocations =
         CarbonDataProcessorUtil.getLocalDataFolderLocation(
             sortParameters.getDatabaseName(), sortParameters.getTableName(),
-            String.valueOf(sortParameters.getTaskNo()), sortParameters.getPartitionID(),
-            sortParameters.getSegmentId() + "", false, false);
+            String.valueOf(sortParameters.getTaskNo()), sortParameters.getSegmentId(),
+            false, false);
     // Set the data file location
     String[] dataFolderLocations = CarbonDataProcessorUtil.arrayAppend(storeLocations,
         File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
index c7030dd..b7452a7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
@@ -133,10 +133,10 @@ public class ParallelReadMergeSorterWithBucketingImpl extends AbstractMergeSorte
   }
 
   private SingleThreadFinalSortFilesMerger getFinalMerger(String bucketId) {
-    String[] storeLocation = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(sortParameters.getDatabaseName(), sortParameters.getTableName(),
-            String.valueOf(sortParameters.getTaskNo()), bucketId,
-            sortParameters.getSegmentId() + "", false, false);
+    String[] storeLocation = CarbonDataProcessorUtil.getLocalDataFolderLocation(
+        sortParameters.getDatabaseName(), sortParameters.getTableName(),
+        String.valueOf(sortParameters.getTaskNo()), sortParameters.getSegmentId(),
+        false, false);
     // Set the data file location
     String[] dataFolderLocation = CarbonDataProcessorUtil.arrayAppend(storeLocation, File.separator,
         CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
@@ -181,10 +181,9 @@ public class ParallelReadMergeSorterWithBucketingImpl extends AbstractMergeSorte
   }
 
   private void setTempLocation(SortParameters parameters) {
-    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(parameters.getDatabaseName(),
-            parameters.getTableName(), parameters.getTaskNo(),
-            parameters.getPartitionID(), parameters.getSegmentId(), false, false);
+    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil.getLocalDataFolderLocation(
+        parameters.getDatabaseName(), parameters.getTableName(), parameters.getTaskNo(),
+        parameters.getSegmentId(), false, false);
     String[] tmpLocs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath, File.separator,
         CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
     parameters.setTempFileLocation(tmpLocs);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
index c5579d9..ed3a55d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
@@ -219,10 +219,9 @@ public class UnsafeBatchParallelReadMergeSorterImpl extends AbstractMergeSorter
     }
 
     private void setTempLocation(SortParameters parameters) {
-      String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
-          .getLocalDataFolderLocation(parameters.getDatabaseName(),
-            parameters.getTableName(), parameters.getTaskNo(), batchCount + "",
-            parameters.getSegmentId(), false, false);
+      String[] carbonDataDirectoryPath = CarbonDataProcessorUtil.getLocalDataFolderLocation(
+          parameters.getDatabaseName(), parameters.getTableName(), parameters.getTaskNo(),
+          parameters.getSegmentId(), false, false);
       String[] tempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
           File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
       parameters.setTempFileLocation(tempDirs);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
index 3c48e4d..f605b22 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
@@ -119,18 +119,17 @@ public class UnsafeParallelReadMergeSorterWithBucketingImpl extends AbstractMerg
 
     Iterator<CarbonRowBatch>[] batchIterator = new Iterator[bucketingInfo.getNumberOfBuckets()];
     for (int i = 0; i < sortDataRows.length; i++) {
-      batchIterator[i] =
-          new MergedDataIterator(String.valueOf(i), batchSize, intermediateFileMergers[i]);
+      batchIterator[i] = new MergedDataIterator(batchSize, intermediateFileMergers[i]);
     }
 
     return batchIterator;
   }
 
-  private UnsafeSingleThreadFinalSortFilesMerger getFinalMerger(String bucketId) {
-    String[] storeLocation = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(sortParameters.getDatabaseName(), sortParameters.getTableName(),
-            String.valueOf(sortParameters.getTaskNo()), bucketId,
-            sortParameters.getSegmentId() + "", false, false);
+  private UnsafeSingleThreadFinalSortFilesMerger getFinalMerger() {
+    String[] storeLocation = CarbonDataProcessorUtil.getLocalDataFolderLocation(
+        sortParameters.getDatabaseName(), sortParameters.getTableName(),
+        String.valueOf(sortParameters.getTaskNo()), sortParameters.getSegmentId(),
+        false, false);
     // Set the data file location
     String[] dataFolderLocation = CarbonDataProcessorUtil.arrayAppend(storeLocation,
         File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
@@ -173,7 +172,7 @@ public class UnsafeParallelReadMergeSorterWithBucketingImpl extends AbstractMerg
   private void setTempLocation(SortParameters parameters) {
     String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
         .getLocalDataFolderLocation(parameters.getDatabaseName(), parameters.getTableName(),
-            parameters.getTaskNo(), parameters.getPartitionID(), parameters.getSegmentId(),
+            parameters.getTaskNo(), parameters.getSegmentId(),
             false, false);
     String[] tmpLoc = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath, File.separator,
         CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
@@ -224,7 +223,6 @@ public class UnsafeParallelReadMergeSorterWithBucketingImpl extends AbstractMerg
 
   private class MergedDataIterator extends CarbonIterator<CarbonRowBatch> {
 
-    private String partitionId;
 
     private int batchSize;
 
@@ -232,9 +230,8 @@ public class UnsafeParallelReadMergeSorterWithBucketingImpl extends AbstractMerg
 
     private UnsafeIntermediateMerger intermediateMerger;
 
-    public MergedDataIterator(String partitionId, int batchSize,
+    public MergedDataIterator(int batchSize,
         UnsafeIntermediateMerger intermediateMerger) {
-      this.partitionId = partitionId;
       this.batchSize = batchSize;
       this.intermediateMerger = intermediateMerger;
       this.firstRow = true;
@@ -245,7 +242,7 @@ public class UnsafeParallelReadMergeSorterWithBucketingImpl extends AbstractMerg
     @Override public boolean hasNext() {
       if (firstRow) {
         firstRow = false;
-        finalMerger = getFinalMerger(partitionId);
+        finalMerger = getFinalMerger();
         List<UnsafeCarbonRowPage> rowPages = intermediateMerger.getRowPages();
         finalMerger.startFinalMerge(rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]),
             intermediateMerger.getMergedPages());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
index 8b87cfc..6cf1dcd 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
@@ -32,6 +32,7 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
@@ -88,11 +89,13 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
     child.initialize();
   }
 
-  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier, String partitionId) {
-    String[] storeLocation = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(tableIdentifier.getDatabaseName(),
-            tableIdentifier.getTableName(), String.valueOf(configuration.getTaskNo()), partitionId,
-            configuration.getSegmentId() + "", false, false);
+  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier) {
+    String[] storeLocation = CarbonDataProcessorUtil.getLocalDataFolderLocation(
+        tableIdentifier.getDatabaseName(),
+        tableIdentifier.getTableName(),
+        String.valueOf(configuration.getTaskNo()), configuration.getSegmentId(),
+        false,
+        false);
     CarbonDataProcessorUtil.createLocations(storeLocation);
     return storeLocation;
   }
@@ -115,11 +118,11 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
       measureCount = configuration.getMeasureCount();
       outputLength = measureCount + (this.noDictWithComplextCount > 0 ? 1 : 0) + 1;
       CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+          .recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
               System.currentTimeMillis());
 
       if (iterators.length == 1) {
-        doExecute(iterators[0], 0, 0);
+        doExecute(iterators[0], 0);
       } else {
         executorService = Executors.newFixedThreadPool(iterators.length,
             new CarbonThreadFactory("NoSortDataWriterPool:" + configuration.getTableIdentifier()
@@ -150,11 +153,10 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
     return null;
   }
 
-  private void doExecute(Iterator<CarbonRowBatch> iterator, int partitionId, int iteratorIndex) {
-    String[] storeLocation = getStoreLocation(tableIdentifier, String.valueOf(partitionId));
-    CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
-        .createCarbonFactDataHandlerModel(configuration, storeLocation, partitionId,
-            iteratorIndex);
+  private void doExecute(Iterator<CarbonRowBatch> iterator, int iteratorIndex) {
+    String[] storeLocation = getStoreLocation(tableIdentifier);
+    CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel.createCarbonFactDataHandlerModel(
+        configuration, storeLocation, 0, iteratorIndex);
     CarbonFactHandler dataHandler = null;
     boolean rowsNotExist = true;
     while (iterator.hasNext()) {
@@ -189,10 +191,11 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
     processingComplete(dataHandler);
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+        .recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
             System.currentTimeMillis());
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
+        .recordMdkGenerateTotalTime(CarbonTablePath.DEPRECATED_PATITION_ID,
+            System.currentTimeMillis());
   }
 
   private void processingComplete(CarbonFactHandler dataHandler) throws CarbonDataLoadingException {
@@ -298,7 +301,7 @@ public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProces
     }
 
     @Override public void run() {
-      doExecute(this.iterator, 0, iteratorIndex);
+      doExecute(this.iterator, iteratorIndex);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bf3602fc/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
index f030d52..369c1f2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.java
@@ -24,6 +24,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
@@ -59,13 +60,11 @@ public class DataWriterBatchProcessorStepImpl extends AbstractDataLoadProcessorS
     child.initialize();
   }
 
-  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier, String partitionId) {
-    String[] storeLocation = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(tableIdentifier.getDatabaseName(),
-            tableIdentifier.getTableName(), String.valueOf(configuration.getTaskNo()), partitionId,
-            configuration.getSegmentId() + "", false, false);
-    CarbonDataProcessorUtil.createLocations(storeLocation);
-    return storeLocation;
+  private String[] getStoreLocation(CarbonTableIdentifier tableIdentifier) {
+    return CarbonDataProcessorUtil.getLocalDataFolderLocation(
+        tableIdentifier.getDatabaseName(), tableIdentifier.getTableName(),
+        String.valueOf(configuration.getTaskNo()),
+        configuration.getSegmentId(), false, false);
   }
 
   @Override public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
@@ -75,18 +74,19 @@ public class DataWriterBatchProcessorStepImpl extends AbstractDataLoadProcessorS
     String tableName = tableIdentifier.getTableName();
     try {
       CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+          .recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
               System.currentTimeMillis());
       int i = 0;
+      String[] storeLocation = getStoreLocation(tableIdentifier);
+      CarbonDataProcessorUtil.createLocations(storeLocation);
       for (Iterator<CarbonRowBatch> iterator : iterators) {
-        String[] storeLocation = getStoreLocation(tableIdentifier, String.valueOf(i));
         int k = 0;
         while (iterator.hasNext()) {
           CarbonRowBatch next = iterator.next();
           // If no rows from merge sorter, then don't create a file in fact column handler
           if (next.hasNext()) {
             CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
-                .createCarbonFactDataHandlerModel(configuration, storeLocation, i, k++);
+                .createCarbonFactDataHandlerModel(configuration, storeLocation, 0, k++);
             CarbonFactHandler dataHandler = CarbonFactHandlerFactory
                 .createCarbonFactHandler(model, CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
             dataHandler.initialise();
@@ -119,10 +119,11 @@ public class DataWriterBatchProcessorStepImpl extends AbstractDataLoadProcessorS
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
     processingComplete(dataHandler);
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+        .recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
             System.currentTimeMillis());
     CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
+        .recordMdkGenerateTotalTime(CarbonTablePath.DEPRECATED_PATITION_ID,
+            System.currentTimeMillis());
   }
 
   private void processingComplete(CarbonFactHandler dataHandler) {


[32/49] carbondata git commit: [REBASE] resolve conflict after rebasing to master

Posted by ja...@apache.org.
[REBASE] resolve conflict after rebasing to master


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

Branch: refs/heads/carbonstore-rebase4
Commit: a6bf77ff2a2bbf36e139581f075a020425754c5b
Parents: 43e34fc
Author: Jacky Li <ja...@qq.com>
Authored: Tue Feb 27 08:51:25 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 08:51:25 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/datamap/TableDataMap.java   |  2 +-
 .../core/datamap/dev/AbstractDataMapWriter.java |  5 ++--
 .../core/datamap/dev/DataMapFactory.java        |  2 +-
 .../core/indexstore/BlockletDetailsFetcher.java |  2 +-
 .../indexstore/SegmentPropertiesFetcher.java    |  3 +-
 .../blockletindex/BlockletDataMap.java          |  2 +-
 .../blockletindex/BlockletDataMapFactory.java   | 21 ++++++-------
 .../core/metadata/SegmentFileStore.java         |  2 +-
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |  1 +
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |  1 +
 ...velRangeLessThanEqualFilterExecuterImpl.java |  1 +
 .../RowLevelRangeLessThanFiterExecuterImpl.java |  1 +
 .../SegmentUpdateStatusManager.java             | 26 ++++------------
 .../apache/carbondata/core/util/CarbonUtil.java | 16 ++++------
 .../testsuite/datamap/CGDataMapTestCase.scala   | 26 ++++++++--------
 .../testsuite/datamap/DataMapWriterSuite.scala  | 19 ++++++------
 .../testsuite/datamap/FGDataMapTestCase.scala   | 31 +++++++++-----------
 .../iud/DeleteCarbonTableTestCase.scala         |  2 +-
 .../TestInsertAndOtherCommandConcurrent.scala   | 14 +++++----
 .../StandardPartitionTableCleanTestCase.scala   | 12 ++++----
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  2 +-
 .../carbondata/spark/util/DataLoadingUtil.scala |  4 +--
 .../CreatePreAggregateTableCommand.scala        |  2 +-
 .../apache/spark/sql/hive/CarbonRelation.scala  |  3 +-
 .../datamap/DataMapWriterListener.java          |  2 +-
 .../loading/model/CarbonLoadModel.java          |  2 +-
 .../processing/merger/CarbonDataMergerUtil.java | 15 +++-------
 .../merger/CompactionResultSortProcessor.java   |  6 ++--
 .../merger/RowResultMergerProcessor.java        |  6 ++--
 .../partition/spliter/RowResultProcessor.java   |  3 +-
 .../util/CarbonDataProcessorUtil.java           |  4 +--
 .../processing/util/CarbonLoaderUtil.java       |  2 +-
 32 files changed, 104 insertions(+), 136 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
index eed650e3..2a6ceaa 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/TableDataMap.java
@@ -143,7 +143,7 @@ public final class TableDataMap extends OperationEventListener {
       blocklets.addAll(
           dataMap.prune(
               filterExp,
-              segmentPropertiesFetcher.getSegmentProperties(distributable.getSegmentId()),
+              segmentPropertiesFetcher.getSegmentProperties(distributable.getSegment()),
               partitions));
     }
     BlockletSerializer serializer = new BlockletSerializer();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
index bcc9bad..de6dcb1 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/AbstractDataMapWriter.java
@@ -18,6 +18,7 @@ package org.apache.carbondata.core.datamap.dev;
 
 import java.io.IOException;
 
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -35,10 +36,10 @@ public abstract class AbstractDataMapWriter {
 
   protected String writeDirectoryPath;
 
-  public AbstractDataMapWriter(AbsoluteTableIdentifier identifier, String segmentId,
+  public AbstractDataMapWriter(AbsoluteTableIdentifier identifier, Segment segment,
       String writeDirectoryPath) {
     this.identifier = identifier;
-    this.segmentId = segmentId;
+    this.segmentId = segment.getSegmentNo();
     this.writeDirectoryPath = writeDirectoryPath;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
index df5670d..50ac279 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapFactory.java
@@ -39,7 +39,7 @@ public interface DataMapFactory<T extends DataMap> {
   /**
    * Return a new write for this datamap
    */
-  AbstractDataMapWriter createWriter(Segment segment);
+  AbstractDataMapWriter createWriter(Segment segment, String writeDirectoryPath);
 
   /**
    * Get the datamap for segmentid

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
index 5a5fc1e..dd592c0 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
@@ -53,5 +53,5 @@ public interface BlockletDetailsFetcher {
    * @param segment
    * @return
    */
-  List<Blocklet> getAllBlocklets(Segment segment, List<String> partitions) throws IOException;
+  List<Blocklet> getAllBlocklets(Segment segment, List<PartitionSpec> partitions) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
index ec2ae93..6f94be5 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/SegmentPropertiesFetcher.java
@@ -19,6 +19,7 @@ package org.apache.carbondata.core.indexstore;
 
 import java.io.IOException;
 
+import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 
 /**
@@ -32,5 +33,5 @@ public interface SegmentPropertiesFetcher {
    * @return
    * @throws IOException
    */
-  SegmentProperties getSegmentProperties(String segmentId) throws IOException;
+  SegmentProperties getSegmentProperties(Segment segment) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
index ef1bd33..ce6193b 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMap.java
@@ -660,7 +660,7 @@ public class BlockletDataMap extends AbstractCoarseGrainDataMap implements Cache
   }
 
   @Override
-  public List<Blocklet> prune(FilterResolverIntf filterExp, List<PartitionSpec> partitions) {
+  public List<Blocklet> prune(FilterResolverIntf filterExp, SegmentProperties segmentProperties, List<PartitionSpec> partitions) {
     if (unsafeMemoryDMStore.getRowCount() == 0) {
       return new ArrayList<>();
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
index 5ca3ac5..ee849bd 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
@@ -38,13 +38,11 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailsFetcher;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.indexstore.SegmentPropertiesFetcher;
 import org.apache.carbondata.core.indexstore.TableBlockIndexUniqueIdentifier;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.SegmentFileStore;
-import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.DataFileFooterConverter;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.events.Event;
 
@@ -57,8 +55,7 @@ import org.apache.hadoop.fs.RemoteIterator;
  * Table map for blocklet
  */
 public class BlockletDataMapFactory extends AbstractCoarseGrainDataMapFactory
-    implements BlockletDetailsFetcher,
-    SegmentPropertiesFetcher {
+    implements BlockletDetailsFetcher, SegmentPropertiesFetcher {
 
   private AbsoluteTableIdentifier identifier;
 
@@ -75,12 +72,12 @@ public class BlockletDataMapFactory extends AbstractCoarseGrainDataMapFactory
   }
 
   @Override
-  public DataMapWriter createWriter(Segment segment) {
+  public AbstractDataMapWriter createWriter(Segment segment, String writeDirectoryPath) {
     throw new UnsupportedOperationException("not implemented");
   }
 
   @Override
-  public List<DataMap> getDataMaps(Segment segment) throws IOException {
+  public List<AbstractCoarseGrainDataMap> getDataMaps(Segment segment) throws IOException {
     List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
         getTableBlockIndexUniqueIdentifiers(segment);
     return cache.getAll(tableBlockIndexUniqueIdentifiers);
@@ -262,8 +259,8 @@ public class BlockletDataMapFactory extends AbstractCoarseGrainDataMapFactory
     return null;
   }
 
-  @Override public SegmentProperties getSegmentProperties(String segmentId) throws IOException {
-    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segmentId);
+  @Override public SegmentProperties getSegmentProperties(Segment segment) throws IOException {
+    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segment);
     assert (dataMaps.size() > 0);
     AbstractCoarseGrainDataMap coarseGrainDataMap = dataMaps.get(0);
     assert (coarseGrainDataMap instanceof BlockletDataMap);
@@ -271,12 +268,12 @@ public class BlockletDataMapFactory extends AbstractCoarseGrainDataMapFactory
     return dataMap.getSegmentProperties();
   }
 
-  @Override public List<Blocklet> getAllBlocklets(String segmentId, List<String> partitions)
+  @Override public List<Blocklet> getAllBlocklets(Segment segment, List<PartitionSpec> partitions)
       throws IOException {
     List<Blocklet> blocklets = new ArrayList<>();
-    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segmentId);
+    List<AbstractCoarseGrainDataMap> dataMaps = getDataMaps(segment);
     for (AbstractCoarseGrainDataMap dataMap : dataMaps) {
-      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segmentId), partitions));
+      blocklets.addAll(dataMap.prune(null, getSegmentProperties(segment), partitions));
     }
     return blocklets;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
index b5f5a25..f48cc6d 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
@@ -432,7 +432,7 @@ public class SegmentFileStore {
       boolean forceDelete) throws IOException {
 
     LoadMetadataDetails[] details =
-        SegmentStatusManager.readLoadMetadata(table.getMetaDataFilepath());
+        SegmentStatusManager.readLoadMetadata(table.getMetadataPath());
     // scan through each segment.
     for (LoadMetadataDetails segment : details) {
       // if this segment is valid then only we will go for deletion of related

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
index 1f63a81..e35bb8a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -32,6 +32,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
index 9140a11..d48abf9 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -32,6 +32,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
index 120671f..d89a488 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -35,6 +35,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
index 547ecaa..a00c7db 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -35,6 +35,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
index 25ce0c8..6ec6fa2 100644
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentUpdateStatusManager.java
@@ -233,10 +233,7 @@ public class SegmentUpdateStatusManager {
    * @throws Exception
    */
   public String[] getDeleteDeltaFilePath(String blockFilePath, String segmentId) throws Exception {
-//    int tableFactPathLength = CarbonTablePath.getFactDir(identifier.getTablePath()).length() + 1;
-//    String blockId = blockFilePath.substring(tableFactPathLength);
-
-    String blockId = CarbonUtil.getBlockId(absoluteTableIdentifier, blockFilePath, segmentId);
+    String blockId = CarbonUtil.getBlockId(identifier, blockFilePath, segmentId);
     String tupleId;
     if (isPartitionTable) {
       tupleId = CarbonTablePath.getShortBlockIdForPartitionTable(blockId);
@@ -249,13 +246,8 @@ public class SegmentUpdateStatusManager {
 
   /**
    * Returns all delta file paths of specified block
-   *
-   * @param tupleId
-   * @param extension
-   * @return
-   * @throws Exception
    */
-  public List<String> getDeltaFiles(String tupleId, String extension) throws Exception {
+  private List<String> getDeltaFiles(String tupleId, String extension) throws Exception {
     try {
       String segment = CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID);
       String completeBlockName = CarbonTablePath.addDataPartPrefix(
@@ -263,11 +255,11 @@ public class SegmentUpdateStatusManager {
               + CarbonCommonConstants.FACT_FILE_EXT);
       String blockPath;
       if (isPartitionTable) {
-        blockPath = absoluteTableIdentifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR
+        blockPath = identifier.getTablePath() + CarbonCommonConstants.FILE_SEPARATOR
             + CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.PART_ID)
             .replace("#", "/") + CarbonCommonConstants.FILE_SEPARATOR + completeBlockName;
       } else {
-        String carbonDataDirectoryPath = carbonTablePath.getCarbonDataDirectoryPath("0", segment);
+        String carbonDataDirectoryPath = CarbonTablePath.getSegmentPath(identifier.getTablePath(), segment);
         blockPath =
             carbonDataDirectoryPath + CarbonCommonConstants.FILE_SEPARATOR + completeBlockName;
       }
@@ -391,16 +383,10 @@ public class SegmentUpdateStatusManager {
    * @return
    */
   public CarbonFile[] getDeleteDeltaFilesList(final Segment segmentId, final String blockName) {
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId.getSegmentNo());
-
+    String segmentPath =
+        CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId.getSegmentNo());
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
-
     for (SegmentUpdateDetails block : updateDetails) {
       if ((block.getBlockName().equalsIgnoreCase(blockName)) &&
           (block.getSegmentName().equalsIgnoreCase(segmentId.getSegmentNo()))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 5ec0158..c9b4337 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -2361,21 +2361,19 @@ public final class CarbonUtil {
   }
 
   // Get the total size of carbon data and the total size of carbon index
-  public static HashMap<String, Long> getDataSizeAndIndexSize(CarbonTablePath carbonTablePath,
+  public static HashMap<String, Long> getDataSizeAndIndexSize(String tablePath,
       Segment segment) throws IOException {
     if (segment.getSegmentFileName() != null) {
-      SegmentFileStore fileStore =
-          new SegmentFileStore(carbonTablePath.getPath(), segment.getSegmentFileName());
+      SegmentFileStore fileStore = new SegmentFileStore(tablePath, segment.getSegmentFileName());
       return getDataSizeAndIndexSize(fileStore);
     } else {
-      return getDataSizeAndIndexSize(carbonTablePath, segment.getSegmentNo());
+      return getDataSizeAndIndexSize(tablePath, segment.getSegmentNo());
     }
   }
 
   // Get the total size of segment.
-  public static long getSizeOfSegment(CarbonTablePath carbonTablePath,
-      Segment segment) throws IOException {
-    HashMap<String, Long> dataSizeAndIndexSize = getDataSizeAndIndexSize(carbonTablePath, segment);
+  public static long getSizeOfSegment(String tablePath, Segment segment) throws IOException {
+    HashMap<String, Long> dataSizeAndIndexSize = getDataSizeAndIndexSize(tablePath, segment);
     long size = 0;
     for (Long eachSize: dataSizeAndIndexSize.values()) {
       size += eachSize;
@@ -2585,9 +2583,7 @@ public final class CarbonUtil {
     String blockName = filePath.substring(filePath.lastIndexOf("/") + 1, filePath.length());
     String tablePath = identifier.getTablePath();
     if (filePath.startsWith(tablePath)) {
-      String factDir =
-          CarbonStorePath.getCarbonTablePath(tablePath, identifier.getCarbonTableIdentifier())
-              .getFactDir();
+      String factDir = CarbonTablePath.getFactDir(tablePath);
       if (filePath.startsWith(factDir)) {
         blockId = "Part0" + CarbonCommonConstants.FILE_SEPARATOR + "Segment_" + segmentId
             + CarbonCommonConstants.FILE_SEPARATOR + blockName;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
index 4b6f231..1cbbcb4 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/CGDataMapTestCase.scala
@@ -27,14 +27,14 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainDataMap, AbstractCoarseGrainDataMapFactory}
 import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMapModel}
-import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
 import org.apache.carbondata.core.datastore.FileReader
 import org.apache.carbondata.core.datastore.block.SegmentProperties
 import org.apache.carbondata.core.datastore.compression.SnappyCompressor
 import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.datastore.page.ColumnPage
-import org.apache.carbondata.core.indexstore.Blocklet
+import org.apache.carbondata.core.indexstore.{Blocklet, PartitionSpec}
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapDistributable
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
 import org.apache.carbondata.core.scan.expression.Expression
@@ -62,16 +62,16 @@ class CGDataMapFactory extends AbstractCoarseGrainDataMapFactory {
   /**
    * Return a new write for this datamap
    */
-  override def createWriter(segmentId: String, dataWritePath: String): AbstractDataMapWriter = {
-    new CGDataMapWriter(identifier, segmentId, dataWritePath, dataMapName)
+  override def createWriter(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
+    new CGDataMapWriter(identifier, segment, dataWritePath, dataMapName)
   }
 
   /**
    * Get the datamap for segmentid
    */
-  override def getDataMaps(segmentId: String): java.util.List[AbstractCoarseGrainDataMap] = {
+  override def getDataMaps(segment: Segment): java.util.List[AbstractCoarseGrainDataMap] = {
     val file = FileFactory.getCarbonFile(
-      CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
 
     val files = file.listFiles(new CarbonFileFilter {
       override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
@@ -108,9 +108,9 @@ class CGDataMapFactory extends AbstractCoarseGrainDataMapFactory {
    *
    * @return
    */
-  override def toDistributable(segmentId: String): java.util.List[DataMapDistributable] = {
+  override def toDistributable(segment: Segment): java.util.List[DataMapDistributable] = {
     val file = FileFactory.getCarbonFile(
-      CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
 
     val files = file.listFiles(new CarbonFileFilter {
       override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
@@ -125,7 +125,7 @@ class CGDataMapFactory extends AbstractCoarseGrainDataMapFactory {
   /**
    * Clears datamap of the segment
    */
-  override def clear(segmentId: String): Unit = {
+  override def clear(segment: Segment): Unit = {
 
   }
 
@@ -175,7 +175,7 @@ class CGDataMap extends AbstractCoarseGrainDataMap {
   override def prune(
       filterExp: FilterResolverIntf,
       segmentProperties: SegmentProperties,
-      partitions: java.util.List[String]): java.util.List[Blocklet] = {
+      partitions: java.util.List[PartitionSpec]): java.util.List[Blocklet] = {
     val buffer: ArrayBuffer[Expression] = new ArrayBuffer[Expression]()
     val expression = filterExp.getFilterExpression
     getEqualToExpression(expression, buffer)
@@ -184,7 +184,7 @@ class CGDataMap extends AbstractCoarseGrainDataMap {
     }
     val meta = findMeta(value(0).getBytes)
     meta.map { f=>
-      new Blocklet(f._1, f._2+"")
+      new Blocklet(f._1, f._2 + "")
     }.asJava
   }
 
@@ -219,10 +219,10 @@ class CGDataMap extends AbstractCoarseGrainDataMap {
 }
 
 class CGDataMapWriter(identifier: AbsoluteTableIdentifier,
-    segmentId: String,
+    segment: Segment,
     dataWritePath: String,
     dataMapName: String)
-  extends AbstractDataMapWriter(identifier, segmentId, dataWritePath) {
+  extends AbstractDataMapWriter(identifier, segment, dataWritePath) {
 
   var currentBlockId: String = null
   val cgwritepath = dataWritePath + "/" +

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index 2f8a1d1..7e93959 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -20,21 +20,19 @@ package org.apache.carbondata.spark.testsuite.datamap
 import java.util
 
 import scala.collection.JavaConverters._
+
 import org.apache.spark.sql.test.util.QueryTest
 import org.apache.spark.sql.{DataFrame, SaveMode}
 import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datamap.dev.{DataMap, DataMapFactory, DataMapWriter}
 import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
-import org.apache.carbondata.core.datamap.dev.AbstractDataMapWriter
+import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMap}
 import org.apache.carbondata.core.datamap.dev.cgdatamap.{AbstractCoarseGrainDataMap, AbstractCoarseGrainDataMapFactory}
-import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
 import org.apache.carbondata.core.datastore.page.ColumnPage
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.scan.filter.intf.ExpressionType
-import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.events.Event
 
@@ -49,15 +47,16 @@ class C2DataMapFactory() extends AbstractCoarseGrainDataMapFactory {
 
   override def fireEvent(event: Event): Unit = ???
 
-  override def clear(segmentId: Segment): Unit = {}
+  override def clear(segment: Segment): Unit = {}
 
   override def clear(): Unit = {}
 
-  override def getDataMaps(distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainDataMap] = ???
+  override def getDataMaps(distributable: DataMapDistributable): util.List[AbstractCoarseGrainDataMap] = ???
 
-  override def getDataMaps(segmentId: Segment): util.List[DataMap] = ???
+  override def getDataMaps(segment: Segment): util.List[AbstractCoarseGrainDataMap] = ???
 
-  override def createWriter(segmentId: Segment): AbstractDataMapWriter = DataMapWriterSuite.dataMapWriterC2Mock
+  override def createWriter(segment: Segment, dataWritePath: String): AbstractDataMapWriter =
+    DataMapWriterSuite.dataMapWriterC2Mock(identifier, segment, dataWritePath)
 
   override def getMeta: DataMapMeta = new DataMapMeta(List("c2").asJava, List(ExpressionType.EQUALS).asJava)
 
@@ -175,9 +174,9 @@ object DataMapWriterSuite {
 
   var callbackSeq: Seq[String] = Seq[String]()
 
-  def dataMapWriterC2Mock(identifier: AbsoluteTableIdentifier, segmentId: String,
+  def dataMapWriterC2Mock(identifier: AbsoluteTableIdentifier, segment: Segment,
       dataWritePath: String) =
-    new AbstractDataMapWriter(identifier, segmentId, dataWritePath) {
+    new AbstractDataMapWriter(identifier, segment, dataWritePath) {
 
     override def onPageAdded(
         blockletId: Int,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
index d1bb65f..9c8cc15 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/FGDataMapTestCase.scala
@@ -27,14 +27,14 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.datamap.dev.fgdatamap.{AbstractFineGrainDataMap, AbstractFineGrainDataMapFactory}
 import org.apache.carbondata.core.datamap.dev.{AbstractDataMapWriter, DataMapModel}
-import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager}
+import org.apache.carbondata.core.datamap.{DataMapDistributable, DataMapMeta, DataMapStoreManager, Segment}
 import org.apache.carbondata.core.datastore.FileReader
 import org.apache.carbondata.core.datastore.block.SegmentProperties
 import org.apache.carbondata.core.datastore.compression.SnappyCompressor
 import org.apache.carbondata.core.datastore.filesystem.{CarbonFile, CarbonFileFilter}
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.datastore.page.ColumnPage
-import org.apache.carbondata.core.indexstore.FineGrainBlocklet
+import org.apache.carbondata.core.indexstore.{Blocklet, FineGrainBlocklet, PartitionSpec}
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapDistributable
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata}
 import org.apache.carbondata.core.scan.expression.Expression
@@ -62,16 +62,16 @@ class FGDataMapFactory extends AbstractFineGrainDataMapFactory {
   /**
    * Return a new write for this datamap
    */
-  override def createWriter(segmentId: String, dataWritePath: String): AbstractDataMapWriter = {
-    new FGDataMapWriter(identifier, segmentId, dataWritePath, dataMapName)
+  override def createWriter(segment: Segment, dataWritePath: String): AbstractDataMapWriter = {
+    new FGDataMapWriter(identifier, segment, dataWritePath, dataMapName)
   }
 
   /**
    * Get the datamap for segmentid
    */
-  override def getDataMaps(segmentId: String): java.util.List[AbstractFineGrainDataMap] = {
+  override def getDataMaps(segment: Segment): java.util.List[AbstractFineGrainDataMap] = {
     val file = FileFactory
-      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
+      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
 
     val files = file.listFiles(new CarbonFileFilter {
       override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
@@ -99,9 +99,9 @@ class FGDataMapFactory extends AbstractFineGrainDataMapFactory {
    *
    * @return
    */
-  override def toDistributable(segmentId: String): java.util.List[DataMapDistributable] = {
-    val file = FileFactory
-      .getCarbonFile(CarbonTablePath.getSegmentPath(identifier.getTablePath, segmentId))
+  override def toDistributable(segment: Segment): java.util.List[DataMapDistributable] = {
+    val file = FileFactory.getCarbonFile(
+      CarbonTablePath.getSegmentPath(identifier.getTablePath, segment.getSegmentNo))
 
     val files = file.listFiles(new CarbonFileFilter {
       override def accept(file: CarbonFile): Boolean = file.getName.endsWith(".datamap")
@@ -112,7 +112,6 @@ class FGDataMapFactory extends AbstractFineGrainDataMapFactory {
     }.toList.asJava
   }
 
-
   /**
    *
    * @param event
@@ -124,7 +123,7 @@ class FGDataMapFactory extends AbstractFineGrainDataMapFactory {
   /**
    * Clears datamap of the segment
    */
-  override def clear(segmentId: String): Unit = {
+  override def clear(segment: Segment): Unit = {
   }
 
   /**
@@ -173,7 +172,7 @@ class FGDataMap extends AbstractFineGrainDataMap {
   override def prune(
       filterExp: FilterResolverIntf,
       segmentProperties: SegmentProperties,
-      partitions: java.util.List[String]): java.util.List[FineGrainBlocklet] = {
+      partitions: java.util.List[PartitionSpec]): java.util.List[Blocklet] = {
     val buffer: ArrayBuffer[Expression] = new ArrayBuffer[Expression]()
     val expression = filterExp.getFilterExpression
     getEqualToExpression(expression, buffer)
@@ -187,7 +186,7 @@ class FGDataMap extends AbstractFineGrainDataMap {
   }
 
   private def readAndFindData(meta: (String, Int, (Array[Byte], Array[Byte]), Long, Int),
-      value: Array[Byte]): Option[FineGrainBlocklet] = {
+      value: Array[Byte]): Option[Blocklet] = {
     val bytes = FileReader.readByteArray(filePath, meta._4, meta._5)
     val outputStream = new ByteArrayInputStream(compressor.unCompressByte(bytes))
     val obj = new ObjectInputStream(outputStream)
@@ -211,12 +210,10 @@ class FGDataMap extends AbstractFineGrainDataMap {
         pg.setRowId(f._2(p._2).toArray)
         pg
       }
-      pages
       Some(new FineGrainBlocklet(meta._1, meta._2.toString, pages.toList.asJava))
     } else {
       None
     }
-
   }
 
   private def findMeta(value: Array[Byte]) = {
@@ -249,8 +246,8 @@ class FGDataMap extends AbstractFineGrainDataMap {
 }
 
 class FGDataMapWriter(identifier: AbsoluteTableIdentifier,
-    segmentId: String, dataWriterPath: String, dataMapName: String)
-  extends AbstractDataMapWriter(identifier, segmentId, dataWriterPath) {
+    segment: Segment, dataWriterPath: String, dataMapName: String)
+  extends AbstractDataMapWriter(identifier, segment, dataWriterPath) {
 
   var currentBlockId: String = null
   val fgwritepath = dataWriterPath + "/" + System.nanoTime() + ".datamap"

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
index 22aa385..f92649a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/DeleteCarbonTableTestCase.scala
@@ -194,7 +194,7 @@ class DeleteCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("delete from update_status_files where age=5").show()
     val carbonTable = CarbonEnv
       .getCarbonTable(Some("iud_db"), "update_status_files")(sqlContext.sparkSession)
-    val metaPath = carbonTable.getMetaDataFilepath
+    val metaPath = carbonTable.getMetadataPath
     val files = FileFactory.getCarbonFile(metaPath)
     assert(files.listFiles().length == 2)
     sql("drop table update_status_files")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
index 5550358..b39c44c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/TestInsertAndOtherCommandConcurrent.scala
@@ -269,7 +269,11 @@ object Global {
 
 class WaitingDataMap() extends AbstractCoarseGrainDataMapFactory {
 
-  override def init(identifier: AbsoluteTableIdentifier, dataMapName: String): Unit = { }
+  private var identifier: AbsoluteTableIdentifier = _
+
+  override def init(identifier: AbsoluteTableIdentifier, dataMapName: String): Unit = {
+    this.identifier = identifier
+  }
 
   override def fireEvent(event: Event): Unit = ???
 
@@ -277,12 +281,12 @@ class WaitingDataMap() extends AbstractCoarseGrainDataMapFactory {
 
   override def clear(): Unit = {}
 
-  override def getDataMaps(distributable: DataMapDistributable): java.util.List[AbstractCoarseGrainDataMap] = ???
+  override def getDataMaps(distributable: DataMapDistributable): util.List[AbstractCoarseGrainDataMap] = ???
 
-  override def getDataMaps(segmentId: Segment): util.List[DataMap] = ???
+  override def getDataMaps(segment: Segment): util.List[AbstractCoarseGrainDataMap] = ???
 
-  override def createWriter(segmentId: Segment): AbstractDataMapWriter = {
-    new AbstractDataMapWriter {
+  override def createWriter(segment: Segment, writeDirectoryPath: String): AbstractDataMapWriter = {
+    new AbstractDataMapWriter(identifier, segment, writeDirectoryPath) {
       override def onPageAdded(blockletId: Int, pageId: Int, pages: Array[ColumnPage]): Unit = { }
 
       override def onBlockletEnd(blockletId: Int): Unit = { }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
index f238d2b..cfc6983 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableCleanTestCase.scala
@@ -52,14 +52,12 @@ class StandardPartitionTableCleanTestCase extends QueryTest with BeforeAndAfterA
 
   def validateDataFiles(tableUniqueName: String, segmentId: String, partition: Int, indexes: Int): Unit = {
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(tableUniqueName)
-    val tablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-      carbonTable.getTablePath)
-    val partitions = CarbonFilters
-      .getPartitions(Seq.empty,
-        sqlContext.sparkSession,
-        TableIdentifier(carbonTable.getTableName, Some(carbonTable.getDatabaseName)))
+    val partitions = CarbonFilters.getPartitions(
+      Seq.empty,
+      sqlContext.sparkSession,
+      TableIdentifier(carbonTable.getTableName, Some(carbonTable.getDatabaseName)))
     assert(partitions.get.length == partition)
-    val details = SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath)
+    val details = SegmentStatusManager.readLoadMetadata(CarbonTablePath.getMetadataPath(carbonTable.getTablePath))
     val segLoad = details.find(_.getLoadName.equals(segmentId)).get
     val seg = new SegmentFileStore(carbonTable.getTablePath, segLoad.getSegmentFile)
     assert(seg.getIndexFiles.size == indexes)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 8ba2767..97e3061 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -291,7 +291,7 @@ class NewCarbonDataLoadRDD[K, V](
         val fileList: java.util.List[String] = new java.util.ArrayList[String](
             CarbonCommonConstants.CONSTANT_SIZE_TEN)
         CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, fileList, ",")
-        model = carbonLoadModel.getCopyWithPartition(
+        model = carbonLoadModel.getCopyWithPartition("0",
           carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
         StandardLogService.setThreadName(StandardLogService
           .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
index cf35c12..49e4420 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -441,8 +441,8 @@ object DataLoadingUtil {
 
   private def isUpdationRequired(isForceDeletion: Boolean,
       carbonTable: CarbonTable,
-      absoluteTableIdentifier: AbsoluteTableIdentifier) = {
-    val details = SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath)
+      absoluteTableIdentifier: AbsoluteTableIdentifier): (Array[LoadMetadataDetails], Boolean) = {
+    val details = SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
     // Delete marked loads
     val isUpdationRequired =
       DeleteLoadFolders.deleteLoadFoldersFromFileSystem(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
index 59c43aa..4d0a4c5 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/CreatePreAggregateTableCommand.scala
@@ -184,7 +184,7 @@ case class CreatePreAggregateTableCommand(
       CarbonFilters.getCurrentPartitions(sparkSession,
       TableIdentifier(parentTable.getTableName,
         Some(parentTable.getDatabaseName))).map(_.asJava).orNull)
-    val loadAvailable = SegmentStatusManager.readLoadMetadata(parentTable.getMetaDataFilepath)
+    val loadAvailable = SegmentStatusManager.readLoadMetadata(parentTable.getMetadataPath)
     if (loadAvailable.exists(load => load.getSegmentStatus == SegmentStatus.INSERT_IN_PROGRESS ||
       load.getSegmentStatus == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) {
       throw new UnsupportedOperationException(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
index c9833d0..5771503 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
@@ -222,9 +222,8 @@ case class CarbonRelation(
           // for each segment calculate the size
           segments.foreach {validSeg =>
             if (validSeg.getSegmentFileName != null) {
-              val fileStore = new SegmentFileStore(tablePath, validSeg.getSegmentFileName)
               size = size + CarbonUtil.getSizeOfSegment(
-                carbonTablePath, new Segment(validSeg.getSegmentNo, validSeg.getSegmentFileName))
+                tablePath, new Segment(validSeg.getSegmentNo, validSeg.getSegmentFileName))
             } else {
               size = size + FileFactory.getDirectorySize(
                 CarbonTablePath.getSegmentPath(tablePath, validSeg.getSegmentNo))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
index 5083ab5..1104229 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datamap/DataMapWriterListener.java
@@ -74,7 +74,7 @@ public class DataMapWriterListener {
     }
     List<String> columns = factory.getMeta().getIndexedColumns();
     List<AbstractDataMapWriter> writers = registry.get(columns);
-    AbstractDataMapWriter writer = factory.createWriter(new Segment(segmentId, null));
+    AbstractDataMapWriter writer = factory.createWriter(new Segment(segmentId, null), dataWritePath);
     if (writers != null) {
       writers.add(writer);
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index a17178a..638ad39 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -478,7 +478,7 @@ public class CarbonLoadModel implements Serializable {
    * @param delimiter
    * @return
    */
-  public CarbonLoadModel getCopyWithPartition(String uniqueId, List<String> filesForPartition,
+  public CarbonLoadModel getCopyWithPartition(String uniqueId,
       String header, String delimiter) {
     CarbonLoadModel copyObj = new CarbonLoadModel();
     copyObj.tableName = tableName;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index 89326a3..d2faef5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -44,7 +44,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
@@ -612,10 +611,10 @@ public final class CarbonDataMergerUtil {
       // variable to store one  segment size across partition.
       long sizeOfOneSegmentAcrossPartition;
       if (segment.getSegmentFile() != null) {
-        sizeOfOneSegmentAcrossPartition = CarbonUtil
-            .getSizeOfSegment(carbonTablePath, new Segment(segId, segment.getSegmentFile()));
+        sizeOfOneSegmentAcrossPartition = CarbonUtil.getSizeOfSegment(
+            tablePath, new Segment(segId, segment.getSegmentFile()));
       } else {
-        sizeOfOneSegmentAcrossPartition = getSizeOfSegment(tablePath, tableIdentifier, segId);
+        sizeOfOneSegmentAcrossPartition = getSizeOfSegment(tablePath, segId);
       }
 
       // if size of a segment is greater than the Major compaction size. then ignore it.
@@ -1006,14 +1005,8 @@ public final class CarbonDataMergerUtil {
   /**
    * This method traverses Update Delta Files inside the seg and return true
    * if UpdateDelta Files are more than IUD Compaction threshold.
-   *
-   * @param seg
-   * @param identifier
-   * @param segmentUpdateStatusManager
-   * @param numberDeltaFilesThreshold
-   * @return
    */
-  public static Boolean checkUpdateDeltaFilesInSeg(Segment seg,
+  private static Boolean checkUpdateDeltaFilesInSeg(Segment seg,
       AbsoluteTableIdentifier absoluteTableIdentifier,
       SegmentUpdateStatusManager segmentUpdateStatusManager, int numberDeltaFilesThreshold) {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index b71612a..ea11e22 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -406,14 +406,12 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
               + carbonLoadModel.getFactTimeStamp() + ".tmp";
     } else {
       carbonStoreLocation = CarbonDataProcessorUtil
-          .createCarbonStoreLocation(carbonTable.getTablePath(), carbonLoadModel.getDatabaseName(),
-              tableName, carbonLoadModel.getPartitionId(), carbonLoadModel.getSegmentId());
+          .createCarbonStoreLocation(carbonLoadModel.getDatabaseName(), tableName, carbonLoadModel.getSegmentId());
     }
     CarbonFactDataHandlerModel carbonFactDataHandlerModel = CarbonFactDataHandlerModel
         .getCarbonFactDataHandlerModel(carbonLoadModel, carbonTable, segmentProperties, tableName,
             tempStoreLocation, carbonStoreLocation);
-    setDataFileAttributesInModel(carbonLoadModel, compactionType, carbonTable,
-        carbonFactDataHandlerModel);
+    setDataFileAttributesInModel(carbonLoadModel, compactionType, carbonFactDataHandlerModel);
     dataHandler = CarbonFactHandlerFactory.createCarbonFactHandler(carbonFactDataHandlerModel,
         CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
index b41829f..278d5bb 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/RowResultMergerProcessor.java
@@ -77,14 +77,12 @@ public class RowResultMergerProcessor extends AbstractResultProcessor {
               .getFactTimeStamp() + ".tmp";
     } else {
       carbonStoreLocation = CarbonDataProcessorUtil
-          .createCarbonStoreLocation(carbonTable.getTablePath(), loadModel.getDatabaseName(),
-              tableName, loadModel.getPartitionId(), loadModel.getSegmentId());
+          .createCarbonStoreLocation(loadModel.getDatabaseName(), tableName, loadModel.getSegmentId());
     }
     CarbonFactDataHandlerModel carbonFactDataHandlerModel = CarbonFactDataHandlerModel
         .getCarbonFactDataHandlerModel(loadModel, carbonTable, segProp, tableName,
             tempStoreLocation, carbonStoreLocation);
-    setDataFileAttributesInModel(loadModel, compactionType, carbonTable,
-        carbonFactDataHandlerModel);
+    setDataFileAttributesInModel(loadModel, compactionType, carbonFactDataHandlerModel);
     carbonFactDataHandlerModel.setCompactionFlow(true);
     dataHandler = new CarbonFactDataHandlerColumnar(carbonFactDataHandlerModel);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
index ff6ca93..df2e2a2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
@@ -48,8 +48,7 @@ public class RowResultProcessor {
     this.segmentProperties = segProp;
     String tableName = carbonTable.getTableName();
     String carbonStoreLocation = CarbonDataProcessorUtil
-        .createCarbonStoreLocation(carbonTable.getTablePath(), loadModel.getDatabaseName(),
-            tableName, loadModel.getPartitionId(), loadModel.getSegmentId());
+        .createCarbonStoreLocation(loadModel.getDatabaseName(), tableName, loadModel.getSegmentId());
     CarbonFactDataHandlerModel carbonFactDataHandlerModel =
         CarbonFactDataHandlerModel.getCarbonFactDataHandlerModel(loadModel, carbonTable,
             segProp, tableName, tempStoreLocation, carbonStoreLocation);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index 2c08c18..dc8ffd7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -34,7 +34,6 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.ColumnType;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
@@ -371,8 +370,7 @@ public final class CarbonDataProcessorUtil {
    *
    * @return data directory path
    */
-  public static String createCarbonStoreLocation(String factStoreLocation,
-      String databaseName, String tableName, String segmentId) {
+  public static String createCarbonStoreLocation(String databaseName, String tableName, String segmentId) {
     CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
     return CarbonTablePath.getSegmentPath(carbonTable.getTablePath(), segmentId);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a6bf77ff/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 1eea61d..a3e889a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -1066,7 +1066,7 @@ public final class CarbonLoaderUtil {
    */
   public static Long addDataIndexSizeIntoMetaEntry(LoadMetadataDetails loadMetadataDetails,
       String segmentId, CarbonTable carbonTable) throws IOException {
-    Map<String, Long> dataIndexSize = CarbonUtil.getDataSizeAndIndexSize(carbonTable.getAbsoluteTableIdentifier(),
+    Map<String, Long> dataIndexSize = CarbonUtil.getDataSizeAndIndexSize(carbonTable.getTablePath(),
         new Segment(segmentId, loadMetadataDetails.getSegmentFile()));
     Long dataSize = dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE);
     loadMetadataDetails.setDataSize(String.valueOf(dataSize));


[19/49] carbondata git commit: [CARBONDATA-2025] Unify all path construction through CarbonTablePath static method

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
index 2a69f0d..a4d3d2b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
@@ -272,7 +272,7 @@ public class CarbonCompactionUtil {
   public static CarbonTable getNextTableToCompact(CarbonTable[] carbonTables,
       List<CarbonTableIdentifier> skipList) {
     for (CarbonTable ctable : carbonTables) {
-      String metadataPath = ctable.getMetaDataFilepath();
+      String metadataPath = ctable.getMetadataPath();
       // check for the compaction required file and at the same time exclude the tables which are
       // present in the skip list.
       if (CarbonCompactionUtil.isCompactionRequiredForTable(metadataPath) && !skipList

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index c141636..89326a3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -169,15 +169,13 @@ public final class CarbonDataMergerUtil {
     // End Timestamp.
 
     // Table Update Status Metadata Update.
-    AbsoluteTableIdentifier absoluteTableIdentifier =
+    AbsoluteTableIdentifier identifier =
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
 
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-
     SegmentUpdateStatusManager segmentUpdateStatusManager =
-        new SegmentUpdateStatusManager(absoluteTableIdentifier);
+        new SegmentUpdateStatusManager(identifier);
 
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
 
     ICarbonLock updateLock = segmentUpdateStatusManager.getTableUpdateStatusLock();
     ICarbonLock statusLock = segmentStatusManager.getTableStatusLock();
@@ -224,7 +222,7 @@ public final class CarbonDataMergerUtil {
           }
 
           LoadMetadataDetails[] loadDetails =
-              segmentStatusManager.readLoadMetadata(metaDataFilepath);
+              SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
           for (LoadMetadataDetails loadDetail : loadDetails) {
             if (loadsToMerge.contains(loadDetail)) {
@@ -237,18 +235,18 @@ public final class CarbonDataMergerUtil {
             }
           }
 
-          segmentUpdateStatusManager
-              .writeLoadDetailsIntoFile(Arrays.asList(updateLists), timestamp);
-          segmentStatusManager
-              .writeLoadDetailsIntoFile(carbonTablePath.getTableStatusFilePath(), loadDetails);
+          segmentUpdateStatusManager.writeLoadDetailsIntoFile(
+              Arrays.asList(updateLists), timestamp);
+          SegmentStatusManager.writeLoadDetailsIntoFile(
+              CarbonTablePath.getTableStatusFilePath(identifier.getTablePath()), loadDetails);
           status = true;
         } else {
           LOGGER.error("Not able to acquire the lock.");
           status = false;
         }
       } catch (IOException e) {
-        LOGGER.error("Error while updating metadata. The metadata file path is " + carbonTablePath
-            .getMetadataDirectoryPath());
+        LOGGER.error("Error while updating metadata. The metadata file path is " +
+            CarbonTablePath.getMetadataPath(identifier.getTablePath()));
         status = false;
 
       } finally {
@@ -284,9 +282,9 @@ public final class CarbonDataMergerUtil {
       String metaDataFilepath, String mergedLoadNumber, CarbonLoadModel carbonLoadModel,
       CompactionType compactionType, String segmentFile) throws IOException {
     boolean tableStatusUpdationStatus = false;
-    AbsoluteTableIdentifier absoluteTableIdentifier =
+    AbsoluteTableIdentifier identifier =
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
 
@@ -295,10 +293,7 @@ public final class CarbonDataMergerUtil {
         LOGGER.info("Acquired lock for the table " + carbonLoadModel.getDatabaseName() + "."
             + carbonLoadModel.getTableName() + " for table status updation ");
 
-        CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier);
-
-        String statusFilePath = carbonTablePath.getTableStatusFilePath();
+        String statusFilePath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
 
         LoadMetadataDetails[] loadDetails = SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
@@ -617,8 +612,6 @@ public final class CarbonDataMergerUtil {
       // variable to store one  segment size across partition.
       long sizeOfOneSegmentAcrossPartition;
       if (segment.getSegmentFile() != null) {
-        CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(carbonTable.getTablePath(), carbonTable.getCarbonTableIdentifier());
         sizeOfOneSegmentAcrossPartition = CarbonUtil
             .getSizeOfSegment(carbonTablePath, new Segment(segId, segment.getSegmentFile()));
       } else {
@@ -662,35 +655,17 @@ public final class CarbonDataMergerUtil {
   /**
    * For calculating the size of the specified segment
    * @param tablePath the store path of the segment
-   * @param tableIdentifier identifier of table that the segment belong to
    * @param segId segment id
    * @return the data size of the segment
    */
-  private static long getSizeOfSegment(String tablePath,
-      CarbonTableIdentifier tableIdentifier, String segId) {
-    String loadPath = getStoreLocation(tablePath, tableIdentifier, segId);
+  private static long getSizeOfSegment(String tablePath, String segId) {
+    String loadPath = CarbonTablePath.getSegmentPath(tablePath, segId);
     CarbonFile segmentFolder =
         FileFactory.getCarbonFile(loadPath, FileFactory.getFileType(loadPath));
     return getSizeOfFactFileInLoad(segmentFolder);
   }
 
   /**
-   * This method will get the store location for the given path, segemnt id and partition id
-   *
-   * @param tablePath
-   * @param carbonTableIdentifier identifier of catbon table that the segment belong to
-   * @param segmentId segment id
-   * @return the store location of the segment
-   */
-  private static String getStoreLocation(String tablePath,
-      CarbonTableIdentifier carbonTableIdentifier, String segmentId) {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(tablePath, carbonTableIdentifier);
-    return carbonTablePath.getCarbonDataDirectoryPath(segmentId);
-  }
-
-
-  /**
    * Identify the segments to be merged based on the segment count
    *
    * @param listOfSegmentsAfterPreserve the list of segments after
@@ -1033,7 +1008,7 @@ public final class CarbonDataMergerUtil {
    * if UpdateDelta Files are more than IUD Compaction threshold.
    *
    * @param seg
-   * @param absoluteTableIdentifier
+   * @param identifier
    * @param segmentUpdateStatusManager
    * @param numberDeltaFilesThreshold
    * @return
@@ -1045,9 +1020,7 @@ public final class CarbonDataMergerUtil {
     CarbonFile[] updateDeltaFiles = null;
     Set<String> uniqueBlocks = new HashSet<String>();
 
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(seg.getSegmentNo());
+    String segmentPath = CarbonTablePath.getSegmentPath(absoluteTableIdentifier.getTablePath(), seg.getSegmentNo());
     CarbonFile segDir =
         FileFactory.getCarbonFile(segmentPath, FileFactory.getFileType(segmentPath));
     CarbonFile[] allSegmentFiles = segDir.listFiles();
@@ -1295,15 +1268,12 @@ public final class CarbonDataMergerUtil {
     CarbonUpdateUtil.updateSegmentStatus(segmentUpdateDetails, table, timestamp, true);
 
     // Update the Table Status.
-    String metaDataFilepath = table.getMetaDataFilepath();
-    AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(absoluteTableIdentifier);
+    String metaDataFilepath = table.getMetadataPath();
+    AbsoluteTableIdentifier identifier = table.getAbsoluteTableIdentifier();
 
-    String tableStatusPath = carbonTablePath.getTableStatusFilePath();
+    String tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
 
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
 
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
 
@@ -1317,7 +1287,7 @@ public final class CarbonDataMergerUtil {
                         + " for table status updation");
 
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-                segmentStatusManager.readLoadMetadata(metaDataFilepath);
+                SegmentStatusManager.readLoadMetadata(metaDataFilepath);
 
         for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
           if (loadMetadata.getLoadName().equalsIgnoreCase("0")) {
@@ -1326,7 +1296,7 @@ public final class CarbonDataMergerUtil {
           }
         }
         try {
-          segmentStatusManager
+          SegmentStatusManager
                   .writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetailsArray);
         } catch (IOException e) {
           return false;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
index 732a7e8..5062a78 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -35,7 +35,6 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.datamap.DataMapWriterListener;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
@@ -337,9 +336,8 @@ public class CarbonFactDataHandlerModel {
       return configuration.getDataWritePath();
     }
     AbsoluteTableIdentifier absoluteTableIdentifier = configuration.getTableIdentifier();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String carbonDataDirectoryPath = carbonTablePath
-        .getCarbonDataDirectoryPath(configuration.getSegmentId() + "");
+    String carbonDataDirectoryPath = CarbonTablePath
+        .getSegmentPath(absoluteTableIdentifier.getTablePath(), configuration.getSegmentId() + "");
     CarbonUtil.checkAndCreateFolder(carbonDataDirectoryPath);
     return carbonDataDirectoryPath;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index e319160..2c08c18 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -44,7 +44,6 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.datatypes.ArrayDataType;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
@@ -143,12 +142,9 @@ public final class CarbonDataProcessorUtil {
     String[] baseTmpStorePathArray = StringUtils.split(baseTempStorePath, File.pathSeparator);
     String[] localDataFolderLocArray = new String[baseTmpStorePathArray.length];
 
-    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
     for (int i = 0 ; i < baseTmpStorePathArray.length; i++) {
       String tmpStore = baseTmpStorePathArray[i];
-      CarbonTablePath carbonTablePath =
-          CarbonStorePath.getCarbonTablePath(tmpStore, carbonTable.getCarbonTableIdentifier());
-      String carbonDataDirectoryPath = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+      String carbonDataDirectoryPath = CarbonTablePath.getSegmentPath(tmpStore, segmentId);
 
       localDataFolderLocArray[i] = carbonDataDirectoryPath + File.separator + taskId;
     }
@@ -376,16 +372,12 @@ public final class CarbonDataProcessorUtil {
    * @return data directory path
    */
   public static String createCarbonStoreLocation(String factStoreLocation,
-      String databaseName, String tableName, String partitionId, String segmentId) {
+      String databaseName, String tableName, String segmentId) {
     CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(databaseName, tableName);
-    CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(factStoreLocation, carbonTableIdentifier);
-    String carbonDataDirectoryPath =
-        carbonTablePath.getCarbonDataDirectoryPath(segmentId);
-    return carbonDataDirectoryPath;
+    return CarbonTablePath.getSegmentPath(carbonTable.getTablePath(), segmentId);
   }
 
+
   /**
    * initialise data type for measures for their storage format
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index c135a88..e7c52f6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -46,7 +46,6 @@ import org.apache.carbondata.core.fileoperations.FileWriteOperation;
 import org.apache.carbondata.core.locks.CarbonLockUtil;
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
@@ -55,7 +54,6 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.merger.NodeBlockRelation;
@@ -74,11 +72,8 @@ public final class CarbonLoaderUtil {
   }
 
   public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
-    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(loadModel.getTablePath(), carbonTable.getCarbonTableIdentifier());
-
-    String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(currentLoad + "");
+    String segmentPath = CarbonTablePath.getSegmentPath(
+        loadModel.getTablePath(), currentLoad + "");
     deleteStorePath(segmentPath);
   }
 
@@ -91,33 +86,26 @@ public final class CarbonLoaderUtil {
    */
   public static boolean isValidSegment(CarbonLoadModel loadModel,
       int currentLoad) {
-    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema()
-        .getCarbonTable();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-        loadModel.getTablePath(), carbonTable.getCarbonTableIdentifier());
 
     int fileCount = 0;
-    int partitionCount = carbonTable.getPartitionCount();
-    for (int i = 0; i < partitionCount; i++) {
-      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(
-          currentLoad + "");
-      CarbonFile carbonFile = FileFactory.getCarbonFile(segmentPath,
-          FileFactory.getFileType(segmentPath));
-      CarbonFile[] files = carbonFile.listFiles(new CarbonFileFilter() {
-
-        @Override
-        public boolean accept(CarbonFile file) {
-          return file.getName().endsWith(
-              CarbonTablePath.getCarbonIndexExtension())
-              || file.getName().endsWith(
-              CarbonTablePath.getCarbonDataExtension());
-        }
-
-      });
-      fileCount += files.length;
-      if (files.length > 0) {
-        return true;
+    String segmentPath = CarbonTablePath.getSegmentPath(
+        loadModel.getTablePath(), currentLoad + "");
+    CarbonFile carbonFile = FileFactory.getCarbonFile(segmentPath,
+        FileFactory.getFileType(segmentPath));
+    CarbonFile[] files = carbonFile.listFiles(new CarbonFileFilter() {
+
+      @Override
+      public boolean accept(CarbonFile file) {
+        return file.getName().endsWith(
+            CarbonTablePath.getCarbonIndexExtension())
+            || file.getName().endsWith(
+            CarbonTablePath.getCarbonDataExtension());
       }
+
+    });
+    fileCount += files.length;
+    if (files.length > 0) {
+      return true;
     }
     if (fileCount == 0) {
       return false;
@@ -183,21 +171,20 @@ public final class CarbonLoaderUtil {
       CarbonLoadModel loadModel, boolean loadStartEntry, boolean insertOverwrite, String uuid,
       List<Segment> segmentsToBeDeleted, List<Segment> segmentFilesTobeUpdated) throws IOException {
     boolean status = false;
-    AbsoluteTableIdentifier absoluteTableIdentifier =
+    AbsoluteTableIdentifier identifier =
         loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String metadataPath = carbonTablePath.getMetadataDirectoryPath();
+    String metadataPath = CarbonTablePath.getMetadataPath(identifier.getTablePath());
     FileType fileType = FileFactory.getFileType(metadataPath);
     if (!FileFactory.isFileExist(metadataPath, fileType)) {
       FileFactory.mkdirs(metadataPath, fileType);
     }
     String tableStatusPath;
     if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildDataMap() && !uuid.isEmpty()) {
-      tableStatusPath = carbonTablePath.getTableStatusFilePathWithUUID(uuid);
+      tableStatusPath = CarbonTablePath.getTableStatusFilePathWithUUID(uuid);
     } else {
-      tableStatusPath = carbonTablePath.getTableStatusFilePath();
+      tableStatusPath = CarbonTablePath.getTableStatusFilePath();
     }
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
     int retryCount = CarbonLockUtil
         .getLockProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
@@ -211,7 +198,8 @@ public final class CarbonLoaderUtil {
             "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
                 + " for table status updation");
         LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-            SegmentStatusManager.readLoadMetadata(carbonTablePath.getMetadataDirectoryPath());
+            SegmentStatusManager.readLoadMetadata(
+                CarbonTablePath.getMetadataPath(identifier.getTablePath()));
         List<LoadMetadataDetails> listOfLoadFolderDetails =
             new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
         List<CarbonFile> staleFolders = new ArrayList<>();
@@ -238,12 +226,12 @@ public final class CarbonLoaderUtil {
           for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
             if (entry.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS
                 && SegmentStatusManager.isLoadInProgress(
-                    absoluteTableIdentifier, entry.getLoadName())) {
+                    identifier, entry.getLoadName())) {
               throw new RuntimeException("Already insert overwrite is in progress");
             } else if (newMetaEntry.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS
                 && entry.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS
                 && SegmentStatusManager.isLoadInProgress(
-                    absoluteTableIdentifier, entry.getLoadName())) {
+                    identifier, entry.getLoadName())) {
               throw new RuntimeException("Already insert into or load is in progress");
             }
           }
@@ -268,7 +256,7 @@ public final class CarbonLoaderUtil {
                 entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
                 // For insert overwrite, we will delete the old segment folder immediately
                 // So collect the old segments here
-                addToStaleFolders(carbonTablePath, staleFolders, entry);
+                addToStaleFolders(identifier, staleFolders, entry);
               }
             }
           }
@@ -281,7 +269,7 @@ public final class CarbonLoaderUtil {
         // when no records are inserted then newSegmentEntry will be SegmentStatus.MARKED_FOR_DELETE
         // so empty segment folder should be deleted
         if (newMetaEntry.getSegmentStatus() == SegmentStatus.MARKED_FOR_DELETE) {
-          addToStaleFolders(carbonTablePath, staleFolders, newMetaEntry);
+          addToStaleFolders(identifier, staleFolders, newMetaEntry);
         }
 
         for (LoadMetadataDetails detail: listOfLoadFolderDetails) {
@@ -326,9 +314,10 @@ public final class CarbonLoaderUtil {
     return status;
   }
 
-  private static void addToStaleFolders(CarbonTablePath carbonTablePath,
+  private static void addToStaleFolders(AbsoluteTableIdentifier identifier,
       List<CarbonFile> staleFolders, LoadMetadataDetails entry) throws IOException {
-    String path = carbonTablePath.getCarbonDataDirectoryPath(entry.getLoadName());
+    String path = CarbonTablePath.getSegmentPath(
+        identifier.getTablePath(), entry.getLoadName());
     // add to the deletion list only if file exist else HDFS file system will throw
     // exception while deleting the file if file path does not exist
     if (FileFactory.isFileExist(path, FileFactory.getFileType(path))) {
@@ -354,11 +343,9 @@ public final class CarbonLoaderUtil {
     loadMetadataDetails.setLoadStartTime(loadStartTime);
   }
 
-  public static void writeLoadMetadata(AbsoluteTableIdentifier absoluteTableIdentifier,
+  public static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
       List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
-    String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
+    String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
 
     DataOutputStream dataOutputStream;
     Gson gsonObjectToWrite = new Gson();
@@ -906,10 +893,8 @@ public final class CarbonLoaderUtil {
    * This method will get the store location for the given path, segment id and partition id
    */
   public static void checkAndCreateCarbonDataLocation(String segmentId, CarbonTable carbonTable) {
-    CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonTable.getTablePath(), carbonTableIdentifier);
-    String segmentFolder = carbonTablePath.getCarbonDataDirectoryPath(segmentId);
+    String segmentFolder = CarbonTablePath.getSegmentPath(
+        carbonTable.getTablePath(), segmentId);
     CarbonUtil.checkAndCreateFolder(segmentFolder);
   }
 
@@ -938,9 +923,7 @@ public final class CarbonLoaderUtil {
    */
   public static Long addDataIndexSizeIntoMetaEntry(LoadMetadataDetails loadMetadataDetails,
       String segmentId, CarbonTable carbonTable) throws IOException {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath((carbonTable.getAbsoluteTableIdentifier()));
-    Map<String, Long> dataIndexSize = CarbonUtil.getDataSizeAndIndexSize(carbonTablePath,
+    Map<String, Long> dataIndexSize = CarbonUtil.getDataSizeAndIndexSize(carbonTable.getAbsoluteTableIdentifier(),
         new Segment(segmentId, loadMetadataDetails.getSegmentFile()));
     Long dataSize = dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE);
     loadMetadataDetails.setDataSize(String.valueOf(dataSize));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
index 288cd54..c00cc86 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
@@ -35,7 +35,6 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 public final class DeleteLoadFolders {
@@ -50,15 +49,14 @@ public final class DeleteLoadFolders {
   /**
    * returns segment path
    *
-   * @param absoluteTableIdentifier
+   * @param identifier
    * @param oneLoad
    * @return
    */
-  private static String getSegmentPath(AbsoluteTableIdentifier absoluteTableIdentifier,
+  private static String getSegmentPath(AbsoluteTableIdentifier identifier,
       LoadMetadataDetails oneLoad) {
-    CarbonTablePath carbon = CarbonStorePath.getCarbonTablePath(absoluteTableIdentifier);
     String segmentId = oneLoad.getLoadName();
-    return carbon.getCarbonDataDirectoryPath(segmentId);
+    return CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
   }
 
   public static void physicalFactAndMeasureMetadataDeletion(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
index cd1e28a..d30891a 100644
--- a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
+++ b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
@@ -63,30 +63,6 @@ public class BlockIndexStoreTest extends TestCase {
 
   }
 
-//  public void testLoadAndGetTaskIdToSegmentsMapForSingleSegment()
-//      throws IOException {
-//    File file = getPartFile();
-//    TableBlockInfo info =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V1, null);
-//    CarbonTableIdentifier carbonTableIdentifier =
-//            new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
-//        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-//    try {
-//
-//      List<TableBlockUniqueIdentifier> tableBlockInfoList =
-//          getTableBlockUniqueIdentifierList(Arrays.asList(new TableBlockInfo[] { info }), absoluteTableIdentifier);
-//      List<AbstractIndex> loadAndGetBlocks = cache.getAll(tableBlockInfoList);
-//      assertTrue(loadAndGetBlocks.size() == 1);
-//    } catch (Exception e) {
-//      assertTrue(false);
-//    }
-//    List<String> segmentIds = new ArrayList<>();
-//      segmentIds.add(info.getSegment());
-//    cache.removeTableBlocks(segmentIds, absoluteTableIdentifier);
-//  }
-//
   private List<TableBlockUniqueIdentifier> getTableBlockUniqueIdentifierList(List<TableBlockInfo> tableBlockInfos,
       AbsoluteTableIdentifier absoluteTableIdentifier) {
     List<TableBlockUniqueIdentifier> tableBlockUniqueIdentifiers = new ArrayList<>();
@@ -95,138 +71,6 @@ public class BlockIndexStoreTest extends TestCase {
     }
     return tableBlockUniqueIdentifiers;
   }
-//
-//  public void testloadAndGetTaskIdToSegmentsMapForSameBlockLoadedConcurrently()
-//      throws IOException {
-//    String canonicalPath =
-//        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
-//    File file = getPartFile();
-//    TableBlockInfo info =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V1, null);
-//    TableBlockInfo info1 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V1, null);
-//
-//    TableBlockInfo info2 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V1, null);
-//    TableBlockInfo info3 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V1, null);
-//    TableBlockInfo info4 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V1, null);
-//
-//    CarbonTableIdentifier carbonTableIdentifier =
-//            new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
-//        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-//    ExecutorService executor = Executors.newFixedThreadPool(3);
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
-//    executor.submit(
-//        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
-//    executor.submit(
-//        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
-//    executor.shutdown();
-//    try {
-//      executor.awaitTermination(1, TimeUnit.DAYS);
-//    } catch (InterruptedException e) {
-//      e.printStackTrace();
-//    }
-//    List<TableBlockInfo> tableBlockInfos =
-//        Arrays.asList(new TableBlockInfo[] { info, info1, info2, info3, info4 });
-//    try {
-//      List<TableBlockUniqueIdentifier> tableBlockUniqueIdentifiers =
-//          getTableBlockUniqueIdentifierList(tableBlockInfos, absoluteTableIdentifier);
-//      List<AbstractIndex> loadAndGetBlocks = cache.getAll(tableBlockUniqueIdentifiers);
-//      assertTrue(loadAndGetBlocks.size() == 5);
-//    } catch (Exception e) {
-//      assertTrue(false);
-//    }
-//    List<String> segmentIds = new ArrayList<>();
-//    for (TableBlockInfo tableBlockInfo : tableBlockInfos) {
-//      segmentIds.add(tableBlockInfo.getSegment());
-//    }
-//    cache.removeTableBlocks(segmentIds, absoluteTableIdentifier);
-//  }
-//
-//  public void testloadAndGetTaskIdToSegmentsMapForDifferentSegmentLoadedConcurrently()
-//      throws IOException {
-//    String canonicalPath =
-//        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
-//    File file = getPartFile();
-//    TableBlockInfo info =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V3, null);
-//    TableBlockInfo info1 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V3, null);
-//
-//    TableBlockInfo info2 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V3, null);
-//    TableBlockInfo info3 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V3, null);
-//    TableBlockInfo info4 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V3, null);
-//
-//    TableBlockInfo info5 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "2", new String[] { "loclhost" },
-//            file.length(),ColumnarFormatVersion.V3, null);
-//    TableBlockInfo info6 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "2", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V3, null);
-//
-//    TableBlockInfo info7 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "3", new String[] { "loclhost" },
-//            file.length(), ColumnarFormatVersion.V3, null);
-//
-//    CarbonTableIdentifier carbonTableIdentifier =
-//            new CarbonTableIdentifier(CarbonCommonConstants.DATABASE_DEFAULT_NAME, "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
-//        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-//    ExecutorService executor = Executors.newFixedThreadPool(3);
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
-//    executor.submit(
-//        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info5, info6 }),
-//        absoluteTableIdentifier));
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info7 }),
-//        absoluteTableIdentifier));
-//
-//    executor.shutdown();
-//    try {
-//      executor.awaitTermination(1, TimeUnit.DAYS);
-//    } catch (InterruptedException e) {
-//      // TODO Auto-generated catch block
-//      e.printStackTrace();
-//    }
-//    List<TableBlockInfo> tableBlockInfos = Arrays
-//        .asList(new TableBlockInfo[] { info, info1, info2, info3, info4, info5, info6, info7 });
-//    try {
-//      List<TableBlockUniqueIdentifier> blockUniqueIdentifierList =
-//          getTableBlockUniqueIdentifierList(tableBlockInfos, absoluteTableIdentifier);
-//      List<AbstractIndex> loadAndGetBlocks = cache.getAll(blockUniqueIdentifierList);
-//      assertTrue(loadAndGetBlocks.size() == 8);
-//    } catch (Exception e) {
-//      assertTrue(false);
-//    }
-//    List<String> segmentIds = new ArrayList<>();
-//    for (TableBlockInfo tableBlockInfo : tableBlockInfos) {
-//      segmentIds.add(tableBlockInfo.getSegment());
-//    }
-//    cache.removeTableBlocks(segmentIds, absoluteTableIdentifier);
-//  }
 
   private class BlockLoaderThread implements Callable<Void> {
     private List<TableBlockInfo> tableBlockInfoList;
@@ -248,7 +92,7 @@ public class BlockIndexStoreTest extends TestCase {
   }
 
   private static File getPartFile() {
-    String path = StoreCreator.getAbsoluteTableIdentifier().getTablePath()
+    String path = StoreCreator.getIdentifier().getTablePath()
         + "/Fact/Part0/Segment_0";
     File file = new File(path);
     File[] files = file.listFiles();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
index 7f0aef6..d42dcde 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
@@ -64,7 +64,6 @@ import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
@@ -98,14 +97,14 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
  */
 public class StoreCreator {
 
-  private static AbsoluteTableIdentifier absoluteTableIdentifier;
+  private static AbsoluteTableIdentifier identifier;
   private static String storePath = "";
   static {
     try {
       storePath = new File("target/store").getCanonicalPath();
       String dbName = "testdb";
       String tableName = "testtable";
-      absoluteTableIdentifier =
+      identifier =
           AbsoluteTableIdentifier.from(
               storePath + "/testdb/testtable",
               new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
@@ -114,8 +113,8 @@ public class StoreCreator {
     }
   }
 
-  public static AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
+  public static AbsoluteTableIdentifier getIdentifier() {
+    return identifier;
   }
 
   /**
@@ -134,12 +133,12 @@ public class StoreCreator {
       CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
       CarbonLoadModel loadModel = new CarbonLoadModel();
       loadModel.setCarbonDataLoadSchema(schema);
-      loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
-      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+      loadModel.setDatabaseName(identifier.getCarbonTableIdentifier().getDatabaseName());
+      loadModel.setTableName(identifier.getCarbonTableIdentifier().getTableName());
+      loadModel.setTableName(identifier.getCarbonTableIdentifier().getTableName());
       loadModel.setFactFilePath(factFilePath);
       loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
-      loadModel.setTablePath(absoluteTableIdentifier.getTablePath());
+      loadModel.setTablePath(identifier.getTablePath());
       loadModel.setDateFormat(null);
       loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
           CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
@@ -175,9 +174,9 @@ public class StoreCreator {
 
   private static CarbonTable createTable() throws IOException {
     TableInfo tableInfo = new TableInfo();
-    tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+    tableInfo.setDatabaseName(identifier.getCarbonTableIdentifier().getDatabaseName());
     TableSchema tableSchema = new TableSchema();
-    tableSchema.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    tableSchema.setTableName(identifier.getCarbonTableIdentifier().getTableName());
     List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
     ArrayList<Encoding> encodings = new ArrayList<>();
     encodings.add(Encoding.DICTIONARY);
@@ -257,16 +256,13 @@ public class StoreCreator {
     tableSchema.setSchemaEvalution(schemaEvol);
     tableSchema.setTableId(UUID.randomUUID().toString());
     tableInfo.setTableUniqueName(
-        absoluteTableIdentifier.getCarbonTableIdentifier().getTableUniqueName()
+        identifier.getCarbonTableIdentifier().getTableUniqueName()
     );
     tableInfo.setLastUpdatedTime(System.currentTimeMillis());
     tableInfo.setFactTable(tableSchema);
-    tableInfo.setTablePath(absoluteTableIdentifier.getTablePath());
+    tableInfo.setTablePath(identifier.getTablePath());
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getTablePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    String schemaFilePath = carbonTablePath.getSchemaFilePath();
+    String schemaFilePath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath());
     String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
     CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
 
@@ -329,7 +325,7 @@ public class StoreCreator {
       writer.close();
       writer.commit();
       Dictionary dict = (Dictionary) dictCache.get(
-          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
+          new DictionaryColumnUniqueIdentifier(identifier,
         		  columnIdentifier, dims.get(i).getDataType()));
       CarbonDictionarySortInfoPreparator preparator =
           new CarbonDictionarySortInfoPreparator();
@@ -444,7 +440,7 @@ public class StoreCreator {
     loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
     listOfLoadFolderDetails.add(loadMetadataDetails);
 
-    String dataLoadLocation = schema.getCarbonTable().getMetaDataFilepath() + File.separator
+    String dataLoadLocation = schema.getCarbonTable().getMetadataPath() + File.separator
         + CarbonCommonConstants.LOADMETADATA_FILENAME;
 
     DataOutputStream dataOutputStream;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
----------------------------------------------------------------------
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
index 7b823ac..8c9889d 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/segment/StreamSegment.java
@@ -38,7 +38,6 @@ import org.apache.carbondata.core.statusmanager.FileFormat;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonIndexFileWriter;
 import org.apache.carbondata.format.BlockIndex;
@@ -60,8 +59,6 @@ public class StreamSegment {
    * get stream segment or create new stream segment if not exists
    */
   public static String open(CarbonTable table) throws IOException {
-    CarbonTablePath tablePath =
-        CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier());
     SegmentStatusManager segmentStatusManager =
         new SegmentStatusManager(table.getAbsoluteTableIdentifier());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
@@ -72,7 +69,8 @@ public class StreamSegment {
                 + " for stream table get or create segment");
 
         LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath());
+            SegmentStatusManager.readLoadMetadata(
+                CarbonTablePath.getMetadataPath(table.getTablePath()));
         LoadMetadataDetails streamSegment = null;
         for (LoadMetadataDetails detail : details) {
           if (FileFormat.ROW_V1 == detail.getFileFormat()) {
@@ -97,8 +95,8 @@ public class StreamSegment {
             newDetails[i] = details[i];
           }
           newDetails[i] = newDetail;
-          SegmentStatusManager
-              .writeLoadDetailsIntoFile(tablePath.getTableStatusFilePath(), newDetails);
+          SegmentStatusManager.writeLoadDetailsIntoFile(
+              CarbonTablePath.getTableStatusFilePath(table.getTablePath()), newDetails);
           return newDetail.getLoadName();
         } else {
           return streamSegment.getLoadName();
@@ -126,8 +124,6 @@ public class StreamSegment {
    */
   public static String close(CarbonTable table, String segmentId)
       throws IOException {
-    CarbonTablePath tablePath =
-        CarbonStorePath.getCarbonTablePath(table.getAbsoluteTableIdentifier());
     SegmentStatusManager segmentStatusManager =
         new SegmentStatusManager(table.getAbsoluteTableIdentifier());
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
@@ -138,7 +134,8 @@ public class StreamSegment {
                 + " for stream table finish segment");
 
         LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath());
+            SegmentStatusManager.readLoadMetadata(
+                CarbonTablePath.getMetadataPath(table.getTablePath()));
         for (LoadMetadataDetails detail : details) {
           if (segmentId.equals(detail.getLoadName())) {
             detail.setLoadEndTime(System.currentTimeMillis());
@@ -162,7 +159,8 @@ public class StreamSegment {
         }
         newDetails[i] = newDetail;
         SegmentStatusManager
-            .writeLoadDetailsIntoFile(tablePath.getTableStatusFilePath(), newDetails);
+            .writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
+                table.getTablePath()), newDetails);
         return newDetail.getLoadName();
       } else {
         LOGGER.error(
@@ -192,7 +190,7 @@ public class StreamSegment {
     try {
       if (statusLock.lockWithRetries()) {
         LoadMetadataDetails[] details =
-            SegmentStatusManager.readLoadMetadata(carbonTable.getMetaDataFilepath());
+            SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
         boolean updated = false;
         for (LoadMetadataDetails detail : details) {
           if (SegmentStatus.STREAMING == detail.getSegmentStatus()) {
@@ -202,10 +200,8 @@ public class StreamSegment {
           }
         }
         if (updated) {
-          CarbonTablePath tablePath =
-              CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier());
           SegmentStatusManager.writeLoadDetailsIntoFile(
-              tablePath.getTableStatusFilePath(),
+              CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath()),
               details);
         }
       } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
index 790f9d8..c9e61d3 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
@@ -38,7 +38,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonProjection}
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.hadoop.streaming.{CarbonStreamInputFormat, CarbonStreamRecordReader}
@@ -221,7 +221,6 @@ object StreamHandoffRDD {
   ): Unit = {
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
     val identifier = carbonTable.getAbsoluteTableIdentifier
-    val tablePath = CarbonStorePath.getCarbonTablePath(identifier)
     var continueHandoff = false
     // require handoff lock on table
     val lock = CarbonLockFactory.getCarbonLockObj(identifier, LockUsage.HANDOFF_LOCK)
@@ -238,7 +237,7 @@ object StreamHandoffRDD {
           try {
             if (statusLock.lockWithRetries()) {
               loadMetadataDetails = SegmentStatusManager.readLoadMetadata(
-                tablePath.getMetadataDirectoryPath)
+                CarbonTablePath.getMetadataPath(identifier.getTablePath))
             }
           } finally {
             if (null != statusLock) {
@@ -360,19 +359,16 @@ object StreamHandoffRDD {
       loadModel: CarbonLoadModel
   ): Boolean = {
     var status = false
-    val metaDataFilepath =
-      loadModel.getCarbonDataLoadSchema().getCarbonTable().getMetaDataFilepath()
-    val identifier =
-      loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier()
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(identifier)
-    val metadataPath = carbonTablePath.getMetadataDirectoryPath()
+    val metaDataFilepath = loadModel.getCarbonDataLoadSchema.getCarbonTable.getMetadataPath
+    val identifier = loadModel.getCarbonDataLoadSchema.getCarbonTable.getAbsoluteTableIdentifier
+    val metadataPath = CarbonTablePath.getMetadataPath(identifier.getTablePath)
     val fileType = FileFactory.getFileType(metadataPath)
     if (!FileFactory.isFileExist(metadataPath, fileType)) {
       FileFactory.mkdirs(metadataPath, fileType)
     }
-    val tableStatusPath = carbonTablePath.getTableStatusFilePath()
+    val tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath)
     val segmentStatusManager = new SegmentStatusManager(identifier)
-    val carbonLock = segmentStatusManager.getTableStatusLock()
+    val carbonLock = segmentStatusManager.getTableStatusLock
     try {
       if (carbonLock.lockWithRetries()) {
         LOGGER.info(
@@ -406,7 +402,7 @@ object StreamHandoffRDD {
         status = true
       } else {
         LOGGER.error("Not able to acquire the lock for Table status updation for table " + loadModel
-          .getDatabaseName() + "." + loadModel.getTableName());
+          .getDatabaseName() + "." + loadModel.getTableName())
       }
     } finally {
       if (carbonLock.unlock()) {
@@ -417,6 +413,6 @@ object StreamHandoffRDD {
                      "." + loadModel.getTableName() + " during table status updation")
       }
     }
-    return status
+    status
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
index 75fcfb0..6316d84 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -31,7 +31,7 @@ import org.apache.carbondata.core.dictionary.service.NonSecureDictionaryServiceP
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
@@ -127,16 +127,14 @@ object StreamSinkFactory {
    * @return
    */
   private def getStreamSegmentId(carbonTable: CarbonTable): String = {
-    val carbonTablePath = CarbonStorePath
-      .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-    val fileType = FileFactory.getFileType(carbonTablePath.getMetadataDirectoryPath)
-    if (!FileFactory.isFileExist(carbonTablePath.getMetadataDirectoryPath, fileType)) {
+    val segmentId = StreamSegment.open(carbonTable)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
+    val fileType = FileFactory.getFileType(segmentDir)
+    if (!FileFactory.isFileExist(segmentDir, fileType)) {
       // Create table directory path, in case of enabling hive metastore first load may not have
       // table folder created.
-      FileFactory.mkdirs(carbonTablePath.getMetadataDirectoryPath, fileType)
+      FileFactory.mkdirs(segmentDir, fileType)
     }
-    val segmentId = StreamSegment.open(carbonTable)
-    val segmentDir = carbonTablePath.getSegmentDir(segmentId)
     if (FileFactory.isFileExist(segmentDir, fileType)) {
       // recover fault
       StreamSegment.recoverSegmentIfRequired(segmentDir)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/4d453d4b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index 206ba91..4f839ce 100644
--- a/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ b/streaming/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -40,7 +40,7 @@ import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.stats.QueryStatistic
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.hadoop.streaming.CarbonStreamOutputFormat
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
@@ -62,9 +62,7 @@ class CarbonAppendableStreamSink(
     carbonLoadModel: CarbonLoadModel,
     server: Option[DictionaryServer]) extends Sink {
 
-  private val carbonTablePath = CarbonStorePath
-    .getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-  private val fileLogPath = carbonTablePath.getStreamingLogDir
+  private val fileLogPath = CarbonTablePath.getStreamingLogDir(carbonTable.getTablePath)
   private val fileLog = new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, fileLogPath)
   // prepare configuration
   private val hadoopConf = {
@@ -150,12 +148,12 @@ class CarbonAppendableStreamSink(
    * if the directory size of current segment beyond the threshold, hand off new segment
    */
   private def checkOrHandOffSegment(): Unit = {
-    val segmentDir = carbonTablePath.getSegmentDir(currentSegmentId)
+    val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
     val fileType = FileFactory.getFileType(segmentDir)
     if (segmentMaxSize <= StreamSegment.size(segmentDir)) {
       val newSegmentId = StreamSegment.close(carbonTable, currentSegmentId)
       currentSegmentId = newSegmentId
-      val newSegmentDir = carbonTablePath.getSegmentDir(currentSegmentId)
+      val newSegmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, currentSegmentId)
       FileFactory.mkdirs(newSegmentDir, fileType)
 
       // TODO trigger hand off operation
@@ -251,15 +249,13 @@ object CarbonAppendableStreamSink {
         }
 
         // update data file info in index file
-        val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-        StreamSegment.updateIndexFile(tablePath.getSegmentDir(segmentId))
+        StreamSegment.updateIndexFile(
+          CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId))
 
       } catch {
         // catch fault of executor side
         case t: Throwable =>
-          val tablePath =
-            CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
-          val segmentDir = tablePath.getSegmentDir(segmentId)
+          val segmentDir = CarbonTablePath.getSegmentPath(carbonTable.getTablePath, segmentId)
           StreamSegment.recoverSegmentIfRequired(segmentDir)
           LOGGER.error(t, s"Aborting job ${ job.getJobID }.")
           committer.abortJob(job)


[03/49] carbondata git commit: [REBASE] Solve conflict after rebasing master

Posted by ja...@apache.org.
[REBASE] Solve conflict after rebasing master


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

Branch: refs/heads/carbonstore-rebase4
Commit: 7f508287629cffdc4e6fe9c35d132229b9894d30
Parents: fc93f11
Author: Jacky Li <ja...@qq.com>
Authored: Thu Feb 1 00:25:31 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Mon Feb 26 23:55:29 2018 +0800

----------------------------------------------------------------------
 .../hadoop/util/CarbonInputFormatUtil.java      | 20 +++++++++++++++++++
 .../spark/rdd/NewCarbonDataLoadRDD.scala        | 21 ++------------------
 .../org/apache/spark/sql/CarbonSession.scala    |  5 ++---
 3 files changed, 24 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f508287/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 514428b..056c27b 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -22,6 +22,8 @@ import java.text.SimpleDateFormat;
 import java.util.List;
 import java.util.Locale;
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
@@ -39,6 +41,7 @@ import org.apache.carbondata.core.scan.model.QueryMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobID;
@@ -159,4 +162,21 @@ public class CarbonInputFormatUtil {
     String jobtrackerID = createJobTrackerID(date);
     return new JobID(jobtrackerID, batch);
   }
+
+  public static void setS3Configurations(Configuration hadoopConf) {
+    FileFactory.getConfiguration()
+        .set("fs.s3a.access.key", hadoopConf.get("fs.s3a.access.key", ""));
+    FileFactory.getConfiguration()
+        .set("fs.s3a.secret.key", hadoopConf.get("fs.s3a.secret.key", ""));
+    FileFactory.getConfiguration()
+        .set("fs.s3a.endpoint", hadoopConf.get("fs.s3a.endpoint", ""));
+    FileFactory.getConfiguration().set(CarbonCommonConstants.S3_ACCESS_KEY,
+        hadoopConf.get(CarbonCommonConstants.S3_ACCESS_KEY, ""));
+    FileFactory.getConfiguration().set(CarbonCommonConstants.S3_SECRET_KEY,
+        hadoopConf.get(CarbonCommonConstants.S3_SECRET_KEY, ""));
+    FileFactory.getConfiguration().set(CarbonCommonConstants.S3N_ACCESS_KEY,
+        hadoopConf.get(CarbonCommonConstants.S3N_ACCESS_KEY, ""));
+    FileFactory.getConfiguration().set(CarbonCommonConstants.S3N_SECRET_KEY,
+        hadoopConf.get(CarbonCommonConstants.S3N_SECRET_KEY, ""));
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f508287/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 917fc88..e17824f 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -41,10 +41,10 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.common.logging.impl.StandardLogService
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
-import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, ThreadLocalTaskInfo}
 import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.processing.loading.{DataLoadExecutor, FailureCauses, TableProcessingOperations}
 import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInputFormat, CSVRecordReaderIterator}
 import org.apache.carbondata.processing.loading.exception.NoRetryException
@@ -371,7 +371,7 @@ class NewDataFrameLoaderRDD[K, V](
   override def internalCompute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
     val hadoopConf = getConf
-    setS3Configurations(hadoopConf)
+    CarbonInputFormatUtil.setS3Configurations(hadoopConf)
     val iter = new Iterator[(K, V)] {
       val loadMetadataDetails = new LoadMetadataDetails()
       val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
@@ -441,23 +441,6 @@ class NewDataFrameLoaderRDD[K, V](
     iter
   }
   override protected def getPartitions: Array[Partition] = firstParent[Row].partitions
-
-  private def setS3Configurations(hadoopConf: Configuration): Unit = {
-    FileFactory.getConfiguration
-      .set("fs.s3a.access.key", hadoopConf.get("fs.s3a.access.key", ""))
-    FileFactory.getConfiguration
-      .set("fs.s3a.secret.key", hadoopConf.get("fs.s3a.secret.key", ""))
-    FileFactory.getConfiguration
-      .set("fs.s3a.endpoint", hadoopConf.get("fs.s3a.endpoint", ""))
-    FileFactory.getConfiguration.set(CarbonCommonConstants.S3_ACCESS_KEY,
-      hadoopConf.get(CarbonCommonConstants.S3_ACCESS_KEY, ""))
-    FileFactory.getConfiguration.set(CarbonCommonConstants.S3_SECRET_KEY,
-      hadoopConf.get(CarbonCommonConstants.S3_SECRET_KEY, ""))
-    FileFactory.getConfiguration.set(CarbonCommonConstants.S3N_ACCESS_KEY,
-      hadoopConf.get(CarbonCommonConstants.S3N_ACCESS_KEY, ""))
-    FileFactory.getConfiguration.set(CarbonCommonConstants.S3N_SECRET_KEY,
-     hadoopConf.get(CarbonCommonConstants.S3N_SECRET_KEY, ""))
-  }
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f508287/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
index 935b0a6..bf958f8 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
@@ -21,7 +21,6 @@ import java.io.File
 import scala.collection.JavaConverters._
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.s3a.Constants.{ACCESS_KEY, ENDPOINT, SECRET_KEY}
 import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
 import org.apache.spark.sql.SparkSession.Builder
@@ -31,8 +30,8 @@ import org.apache.spark.sql.internal.{SessionState, SharedState}
 import org.apache.spark.util.{CarbonReflectionUtils, Utils}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, ThreadLocalSessionInfo}
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 
 /**
  * Session implementation for {org.apache.spark.sql.SparkSession}
@@ -154,7 +153,7 @@ object CarbonSession {
             sparkConf.setAppName(randomAppName)
           }
           val sc = SparkContext.getOrCreate(sparkConf)
-          setS3Configurations(sc)
+          CarbonInputFormatUtil.setS3Configurations(sc.hadoopConfiguration)
           // maybe this is an existing SparkContext, update its SparkConf which maybe used
           // by SparkSession
           options.foreach { case (k, v) => sc.conf.set(k, v) }


[07/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index f4450e3..5f8d199 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -31,7 +31,7 @@ import java.util.Map;
 
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
 import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -45,7 +45,7 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 
 import mockit.Mock;
 import mockit.MockUp;
@@ -267,8 +267,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValue() {
     byte[] dataChunks = { 5, 6, 7, 8, 9 };
     byte[] compareValues = { 7 };
-    FixedLengthDimensionDataChunk fixedLengthDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+    FixedLengthDimensionColumnPage fixedLengthDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
     int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, compareValues);
     assertEquals(result, 1);
   }
@@ -276,8 +276,8 @@ public class CarbonUtilTest {
   @Test public void testToGetNextLesserValueToTarget() {
     byte[] dataChunks = { 7, 7, 7, 8, 9 };
     byte[] compareValues = { 7 };
-    FixedLengthDimensionDataChunk fixedLengthDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+    FixedLengthDimensionColumnPage fixedLengthDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
     int result = CarbonUtil.nextLesserValueToTarget(2, fixedLengthDataChunk, compareValues);
     assertEquals(result, -1);
   }
@@ -285,8 +285,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValue() {
     byte[] dataChunks = { 5, 6, 7, 8, 9 };
     byte[] compareValues = { 7 };
-    FixedLengthDimensionDataChunk fixedLengthDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+    FixedLengthDimensionColumnPage fixedLengthDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
     int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, compareValues, 5);
     assertEquals(result, 3);
   }
@@ -302,8 +302,8 @@ public class CarbonUtilTest {
   @Test public void testToGetnextGreaterValueToTarget() {
     byte[] dataChunks = { 5, 6, 7, 7, 7 };
     byte[] compareValues = { 7 };
-    FixedLengthDimensionDataChunk fixedLengthDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 5, 1);
+    FixedLengthDimensionColumnPage fixedLengthDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 5, 1);
     int result = CarbonUtil.nextGreaterValueToTarget(2, fixedLengthDataChunk, compareValues, 5);
     assertEquals(result, 5);
   }
@@ -525,23 +525,23 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetDictionaryEncodingArray() {
-    QueryDimension column1 = new QueryDimension("Column1");
-    QueryDimension column2 = new QueryDimension("Column2");
     ColumnSchema column1Schema = new ColumnSchema();
     ColumnSchema column2Schema = new ColumnSchema();
     column1Schema.setColumnName("Column1");
     List<Encoding> encoding = new ArrayList<>();
     encoding.add(Encoding.DICTIONARY);
     column1Schema.setEncodingList(encoding);
-    column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
 
     column2Schema.setColumnName("Column2");
     List<Encoding> encoding2 = new ArrayList<>();
     encoding2.add(Encoding.DELTA);
     column2Schema.setEncodingList(encoding2);
-    column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
 
-    QueryDimension[] queryDimensions = { column1, column2 };
+    ProjectionDimension[] queryDimensions = { column1, column2 };
 
     boolean[] dictionaryEncoding = CarbonUtil.getDictionaryEncodingArray(queryDimensions);
     boolean[] expectedDictionaryEncoding = { true, false };
@@ -551,23 +551,23 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetDirectDictionaryEncodingArray() {
-    QueryDimension column1 = new QueryDimension("Column1");
-    QueryDimension column2 = new QueryDimension("Column2");
     ColumnSchema column1Schema = new ColumnSchema();
     ColumnSchema column2Schema = new ColumnSchema();
     column1Schema.setColumnName("Column1");
     List<Encoding> encoding = new ArrayList<>();
     encoding.add(Encoding.DIRECT_DICTIONARY);
     column1Schema.setEncodingList(encoding);
-    column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
 
     column2Schema.setColumnName("Column2");
     List<Encoding> encoding2 = new ArrayList<>();
     encoding2.add(Encoding.DELTA);
     column2Schema.setEncodingList(encoding2);
-    column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
 
-    QueryDimension[] queryDimensions = { column1, column2 };
+    ProjectionDimension[] queryDimensions = { column1, column2 };
 
     boolean[] dictionaryEncoding = CarbonUtil.getDirectDictionaryEncodingArray(queryDimensions);
     boolean[] expectedDictionaryEncoding = { true, false };
@@ -577,19 +577,19 @@ public class CarbonUtilTest {
   }
 
   @Test public void testToGetComplexDataTypeArray() {
-    QueryDimension column1 = new QueryDimension("Column1");
-    QueryDimension column2 = new QueryDimension("Column2");
     ColumnSchema column1Schema = new ColumnSchema();
     ColumnSchema column2Schema = new ColumnSchema();
     column1Schema.setColumnName("Column1");
     column1Schema.setDataType(DataTypes.DATE);
-    column1.setDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column1 = new ProjectionDimension(new CarbonDimension(column1Schema, 1, 1, 1, 1));
 
     column2Schema.setColumnName("Column2");
     column2Schema.setDataType(DataTypes.createDefaultArrayType());
-    column2.setDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
+    ProjectionDimension
+        column2 = new ProjectionDimension(new CarbonDimension(column2Schema, 1, 1, 1, 1));
 
-    QueryDimension[] queryDimensions = { column1, column2 };
+    ProjectionDimension[] queryDimensions = { column1, column2 };
 
     boolean[] dictionaryEncoding = CarbonUtil.getComplexDataTypeArray(queryDimensions);
     boolean[] expectedDictionaryEncoding = { false, true };
@@ -806,8 +806,8 @@ public class CarbonUtilTest {
   @Test public void testToGetFirstIndexUsingBinarySearchWithCompareTo1() {
     byte[] dataChunks = { 10, 20, 30, 40, 50, 60 };
     byte[] compareValue = { 5 };
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1);
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1);
     int result = CarbonUtil
         .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false);
     assertEquals(-2, result);
@@ -816,8 +816,8 @@ public class CarbonUtilTest {
   @Test public void testToGetFirstIndexUsingBinarySearchWithCompareToLessThan0() {
     byte[] dataChunks = { 10, 20, 30, 40, 50, 60 };
     byte[] compareValue = { 30 };
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1);
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1);
     int result = CarbonUtil
         .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false);
     assertEquals(2, result);
@@ -826,8 +826,8 @@ public class CarbonUtilTest {
   @Test public void testToGetFirstIndexUsingBinarySearchWithCompareTo0() {
     byte[] dataChunks = { 10, 10, 10, 40, 50, 60 };
     byte[] compareValue = { 10 };
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1);
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1);
     int result = CarbonUtil
         .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, false);
     assertEquals(0, result);
@@ -836,8 +836,8 @@ public class CarbonUtilTest {
   @Test public void testToGetFirstIndexUsingBinarySearchWithMatchUpLimitTrue() {
     byte[] dataChunks = { 10, 10, 10, 40, 50, 60 };
     byte[] compareValue = { 10 };
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(dataChunks, null, null, 6, 1);
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk =
+        new FixedLengthDimensionColumnPage(dataChunks, null, null, 6, 1);
     int result = CarbonUtil
         .getFirstIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 1, 3, compareValue, true);
     assertEquals(2, result);
@@ -847,13 +847,13 @@ public class CarbonUtilTest {
   public void testBinaryRangeSearch() {
 
     byte[] dataChunk = new byte[10];
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
     byte[] keyWord = new byte[1];
     int[] range;
 
     dataChunk = "abbcccddddeffgggh".getBytes();
     byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length];
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) {
@@ -885,7 +885,7 @@ public class CarbonUtilTest {
     assertRangeIndex(dataArr, dataChunk, fixedLengthDimensionDataChunk, keyWord, expectRangeIndex);
 
     dataChunk = "ab".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     keyWord[0] = Byte.valueOf("97");
@@ -899,7 +899,7 @@ public class CarbonUtilTest {
     assertEquals(1, range[1]);
 
     dataChunk = "aabb".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     keyWord[0] = Byte.valueOf("97");
@@ -913,7 +913,7 @@ public class CarbonUtilTest {
     assertEquals(3, range[1]);
 
     dataChunk = "a".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     keyWord[0] = Byte.valueOf("97");
@@ -922,7 +922,7 @@ public class CarbonUtilTest {
     assertEquals(0, range[1]);
 
     dataChunk = "aa".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     keyWord[0] = Byte.valueOf("97");
@@ -931,7 +931,7 @@ public class CarbonUtilTest {
     assertEquals(1, range[1]);
 
     dataChunk = "aabbbbbbbbbbcc".getBytes();
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
     keyWord[0] = Byte.valueOf("98");
     range = CarbonUtil.getRangeIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 0, dataChunk.length - 1, keyWord);
@@ -944,14 +944,14 @@ public class CarbonUtilTest {
   public void IndexUsingBinarySearchLengthTwo() {
 
     byte[] dataChunk = new byte[10];
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
 
     byte[] keyWord = new byte[2];
 
     dataChunk = "aabbbbbbbbbbcc".getBytes();
     byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length];
 
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) {
@@ -986,14 +986,14 @@ public class CarbonUtilTest {
   public void IndexUsingBinarySearchLengthThree() {
 
     byte[] dataChunk = new byte[10];
-    FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+    FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
 
     byte[] keyWord = new byte[3];
 
     dataChunk = "aaabbbbbbbbbccc".getBytes();
     byte[][] dataArr = new byte[dataChunk.length / keyWord.length][keyWord.length];
 
-    fixedLengthDimensionDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    fixedLengthDimensionDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / keyWord.length, keyWord.length);
 
     for (int ii = 0; ii < dataChunk.length / keyWord.length; ii++) {
@@ -1101,7 +1101,7 @@ public class CarbonUtilTest {
   }
 
   private void assertRangeIndex(byte[][] dataArr, byte[] dataChunk,
-      FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk, byte[] keyWord, int[] expectRangeIndex) {
+      FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk, byte[] keyWord, int[] expectRangeIndex) {
     int[] range;
     range = CarbonUtil.getRangeIndexUsingBinarySearch(fixedLengthDimensionDataChunk, 0,
         (dataChunk.length - 1) / keyWord.length, keyWord);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
index de64c0a..e506994 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/DataFileFooterConverterTest.java
@@ -25,10 +25,10 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.datastore.impl.FileHolderImpl;
+import org.apache.carbondata.core.datastore.impl.FileReaderImpl;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.SegmentInfo;
@@ -229,13 +229,13 @@ public class DataFileFooterConverterTest {
       }
 
       @SuppressWarnings("unused") @Mock
-      public FileHolder getFileHolder(FileFactory.FileType fileType) {
-        return new FileHolderImpl();
+      public FileReader getFileHolder(FileFactory.FileType fileType) {
+        return new FileReaderImpl();
       }
 
     };
 
-    new MockUp<FileHolderImpl>() {
+    new MockUp<FileReaderImpl>() {
       @SuppressWarnings("unused") @Mock public long readLong(String filePath, long offset) {
         return 1;
       }
@@ -249,7 +249,6 @@ public class DataFileFooterConverterTest {
     SegmentInfo segmentInfo = new SegmentInfo();
     int[] arr = { 1, 2, 3 };
     segmentInfo.setColumnCardinality(arr);
-    segmentInfo.setNumberOfColumns(segmentInfo1.getNum_cols());
     dataFileFooter.setNumberOfRows(3);
     dataFileFooter.setSegmentInfo(segmentInfo);
     TableBlockInfo info = new TableBlockInfo("/file.carbondata", 1, "0", new String[0], 1, ColumnarFormatVersion.V1, null);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
index 4c9a784..4fb5dcc 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/RangeFilterProcessorTest.java
@@ -36,7 +36,6 @@ import org.apache.carbondata.core.scan.expression.logical.RangeExpression;
 import org.apache.carbondata.core.scan.expression.logical.TrueExpression;
 import org.apache.carbondata.core.scan.filter.executer.RangeValueFilterExecuterImpl;
 import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer;
-import org.apache.carbondata.core.scan.filter.intf.FilterOptimizerBasic;
 import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer;
 
 import mockit.Deencapsulation;
@@ -102,7 +101,7 @@ public class RangeFilterProcessorTest {
         new LessThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING),
             new LiteralExpression("20", DataTypes.STRING))), new TrueExpression(null));
     FilterOptimizer rangeFilterOptimizer =
-        new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
+        new RangeFilterOptmizer(inputFilter);
     rangeFilterOptimizer.optimizeFilter();
     result = checkBothTrees(inputFilter, output);
     Assert.assertTrue(result);
@@ -143,7 +142,7 @@ public class RangeFilterProcessorTest {
         new LessThanEqualToExpression(new ColumnExpression("a", DataTypes.STRING),
             new LiteralExpression("05", DataTypes.STRING)));
     FilterOptimizer rangeFilterOptimizer =
-        new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
+        new RangeFilterOptmizer(inputFilter);
     rangeFilterOptimizer.optimizeFilter();
     result = checkBothTrees(inputFilter, output);
     // no change
@@ -218,7 +217,7 @@ public class RangeFilterProcessorTest {
     Expression Andb3 = new AndExpression(Andb2, new TrueExpression(null));
 
     FilterOptimizer rangeFilterOptimizer =
-        new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
+        new RangeFilterOptmizer(inputFilter);
     rangeFilterOptimizer.optimizeFilter();
     result = checkBothTrees(inputFilter, new AndExpression(Andb3, new TrueExpression(null)));
     // no change
@@ -302,7 +301,7 @@ public class RangeFilterProcessorTest {
     Expression Orb3 = new OrExpression(Orb2, lessThanb2);
 
     FilterOptimizer rangeFilterOptimizer =
-        new RangeFilterOptmizer(new FilterOptimizerBasic(), inputFilter);
+        new RangeFilterOptmizer(inputFilter);
     rangeFilterOptimizer.optimizeFilter();
     result = checkBothTrees(inputFilter, new OrExpression(Orb3, lessThanb1));
     // no change

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java b/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
deleted file mode 100644
index 94c3f68..0000000
--- a/core/src/test/java/org/apache/carbondata/scanner/impl/FilterScannerTest.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.scanner.impl;
-
-import static junit.framework.TestCase.assertEquals;
-
-public class FilterScannerTest {
-//
-//  private static FilterScanner filterScanner;
-//  private static BlockletIndex blockletIndex;
-//  private static BlockletMinMaxIndex blockletMinMaxIndex;
-//  private static BTreeBuilderInfo bTreeBuilderInfo;
-//  private static DataFileFooter dataFileFooter;
-//
-//  @BeforeClass public static void setUp() {
-//    BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
-//    FilterExecuter filterExecutor = new AndFilterExecuterImpl(null, null);
-//    blockExecutionInfo.setFilterExecuterTree(filterExecutor);
-//    blockExecutionInfo.setFixedLengthKeySize(1);
-//    blockExecutionInfo.setNoDictionaryBlockIndexes(new int[] { 1, 2 });
-//    blockExecutionInfo.setDictionaryColumnBlockIndex(new int[] { 1 });
-//    blockExecutionInfo.setColumnGroupToKeyStructureInfo(new HashMap<Integer, KeyStructureInfo>());
-//    blockExecutionInfo.setComplexDimensionInfoMap(new HashMap<Integer, GenericQueryType>());
-//    blockExecutionInfo.setComplexColumnParentBlockIndexes(new int[] { 1 });
-//    blockExecutionInfo.setQueryDimensions(new QueryDimension[] { new QueryDimension("Col1") });
-//    blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(new int[][] { { 0, 0 } });
-//    blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(new int[][] { { 0, 0 } });
-//    blockExecutionInfo.setTotalNumberOfMeasureBlock(1);
-//    blockExecutionInfo.setTotalNumberDimensionBlock(1);
-//    QueryStatisticsModel queryStatisticsModel = new QueryStatisticsModel();
-//    QueryStatistic queryStatistic = new QueryStatistic();
-//    queryStatistic.addCountStatistic(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, 1);
-//    Map<String, QueryStatistic> statisticsTypeAndObjMap = new HashMap<>();
-//    statisticsTypeAndObjMap.put(QueryStatisticsConstants.TOTAL_BLOCKLET_NUM, queryStatistic);
-//    statisticsTypeAndObjMap.put(QueryStatisticsConstants.VALID_SCAN_BLOCKLET_NUM, queryStatistic);
-//    queryStatisticsModel.setStatisticsTypeAndObjMap(statisticsTypeAndObjMap);
-//    QueryStatisticsRecorder queryStatisticsRecorder = new QueryStatisticsRecorderImpl("1");
-//    queryStatisticsModel.setRecorder(queryStatisticsRecorder);
-//    filterScanner = new FilterScanner(blockExecutionInfo, queryStatisticsModel);
-//    blockletIndex = new BlockletIndex();
-//    blockletMinMaxIndex = new BlockletMinMaxIndex();
-//    blockletMinMaxIndex.setMinValues(new byte[][] { { 1, 2 } });
-//    blockletMinMaxIndex.setMaxValues(new byte[][] { { 10, 12 } });
-//    blockletIndex.setMinMaxIndex(blockletMinMaxIndex);
-//    dataFileFooter = new DataFileFooter();
-//    dataFileFooter.setBlockletIndex(blockletIndex);
-//    bTreeBuilderInfo = new BTreeBuilderInfo(Arrays.asList(dataFileFooter), new int[] { 1 });
-//  }
-//
-//  @Test public void testToScanBlockletWithEmptyBitSet() throws QueryExecutionException {
-//    new MockUp<AndFilterExecuterImpl>() {
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-//        return new BitSet();
-//      }
-//    };
-//    BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1);
-//    DataRefNode dataRefNode = new BlockBTreeLeafNode(bTreeBuilderInfo, 0, 1);
-//    blocksChunkHolder.setDataBlock(dataRefNode);
-//    AbstractScannedResult abstractScannedResult = filterScanner.scanBlocklet(blocksChunkHolder);
-//    assertEquals(0, abstractScannedResult.numberOfOutputRows());
-//  }
-//
-//  @Test public void testToScanBlockletWithNonEmptyBitSet() throws QueryExecutionException {
-//    new MockUp<AndFilterExecuterImpl>() {
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-//        BitSet bitSet = new BitSet();
-//        bitSet.set(1);
-//        bitSet.set(2);
-//        bitSet.set(1);
-//        return bitSet;
-//      }
-//
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
-//          throws FilterUnsupportedException {
-//        BitSet bitSet = new BitSet();
-//        bitSet.set(1);
-//        bitSet.set(2);
-//        bitSet.set(1);
-//        return bitSet;
-//      }
-//    };
-//    DataRefNode dataRefNode = new MockUp<DataRefNode>() {
-//      @Mock @SuppressWarnings("unused") DimensionColumnDataChunk[] getDimensionChunks(
-//          FileHolder fileReader, int[][] blockIndexes) {
-//        DimensionColumnDataChunk[] dimensionChunkAttributes =
-//            { new ColumnGroupDimensionDataChunk(null, null) };
-//        return dimensionChunkAttributes;
-//      }
-//
-//      @Mock @SuppressWarnings("unused") ColumnPage[] getMeasureChunks(
-//          FileHolder fileReader, int[][] blockIndexes) {
-//
-//        ColumnPage[] ColumnPages = { new ColumnPage() };
-//        return ColumnPages;
-//      }
-//    }.getMockInstance();
-//
-//    BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1);
-//    blocksChunkHolder.setDataBlock(dataRefNode);
-//    DimensionChunkAttributes dimensionChunkAttributes = new DimensionChunkAttributes();
-//    DimensionColumnDataChunk dimensionColumnDataChunk =
-//        new FixedLengthDimensionDataChunk(new byte[] { 0, 1 }, dimensionChunkAttributes);
-//    blocksChunkHolder.setDimensionRawDataChunk(new DimensionColumnDataChunk[]
-//
-//        { dimensionColumnDataChunk });
-//    ColumnPage ColumnPage = new ColumnPage();
-//    blocksChunkHolder.setMeasureDataChunk(new ColumnPage[]
-//
-//        { ColumnPage });
-//    FileHolder fileHolder = new DFSFileHolderImpl();
-//    blocksChunkHolder.setFileReader(fileHolder);
-//    AbstractScannedResult abstractScannedResult = filterScanner.scanBlocklet(blocksChunkHolder);
-//
-//    assertEquals(2, abstractScannedResult.numberOfOutputRows());
-//  }
-//
-//  @Test(expected = QueryExecutionException.class) public void testToScanBlockletWithException()
-//      throws QueryExecutionException {
-//    new MockUp<AndFilterExecuterImpl>() {
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-//        BitSet bitSet = new BitSet();
-//        bitSet.set(1);
-//        bitSet.set(2);
-//        bitSet.set(1);
-//        return bitSet;
-//      }
-//
-//      @SuppressWarnings("unused") @Mock
-//      public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
-//          throws FilterUnsupportedException {
-//        throw new FilterUnsupportedException("Filter unsupported");
-//      }
-//    };
-//    BlocksChunkHolder blocksChunkHolder = new BlocksChunkHolder(1, 1);
-//    BTreeBuilderInfo bTreeBuilderInfo =
-//        new BTreeBuilderInfo(Arrays.asList(dataFileFooter), new int[] { 1 });
-//    DataRefNode dataRefNode = new BlockBTreeLeafNode(bTreeBuilderInfo, 0, 1);
-//    blocksChunkHolder.setDataBlock(dataRefNode);
-//    filterScanner.scanBlocklet(blocksChunkHolder);
-//  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/dev/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/dev/findbugs-exclude.xml b/dev/findbugs-exclude.xml
index 1520cd4..b19db85 100644
--- a/dev/findbugs-exclude.xml
+++ b/dev/findbugs-exclude.xml
@@ -31,7 +31,7 @@
     <Bug pattern="OBL_UNSATISFIED_OBLIGATION_EXCEPTION_EDGE"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datastore.impl.FileHolderImpl"/>
+    <Class name="org.apache.carbondata.core.datastore.impl.FileReaderImpl"/>
     <Method name="getDataInputStream"/>
     <Bug pattern="OBL_UNSATISFIED_OBLIGATION_EXCEPTION_EDGE"/>
   </Match>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
index 76afcbf..7a15327 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
@@ -62,7 +62,7 @@ object CarbonSessionExample {
 
     spark.sql(
       s"""
-         | SELECT *
+         | SELECT charField, stringField, intField
          | FROM carbon_table
          | WHERE stringfield = 'spark' AND decimalField > 40
       """.stripMargin).show()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 069e1f7..087cf55 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -58,7 +58,6 @@ import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
 import org.apache.carbondata.core.scan.filter.TableProvider;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
@@ -111,11 +110,11 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   // comma separated list of input segment numbers
   public static final String INPUT_SEGMENT_NUMBERS =
       "mapreduce.input.carboninputformat.segmentnumbers";
-  public static final String VALIDATE_INPUT_SEGMENT_IDs =
+  private static final String VALIDATE_INPUT_SEGMENT_IDs =
       "mapreduce.input.carboninputformat.validsegments";
   // comma separated list of input files
   public static final String INPUT_FILES = "mapreduce.input.carboninputformat.files";
-  public static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
+  private static final String ALTER_PARTITION_ID = "mapreduce.input.carboninputformat.partitionid";
   private static final Log LOG = LogFactory.getLog(CarbonTableInputFormat.class);
   private static final String FILTER_PREDICATE =
       "mapreduce.input.carboninputformat.filter.predicate";
@@ -126,7 +125,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   private static final String DATA_MAP_DSTR = "mapreduce.input.carboninputformat.datamapdstr";
   public static final String DATABASE_NAME = "mapreduce.input.carboninputformat.databaseName";
   public static final String TABLE_NAME = "mapreduce.input.carboninputformat.tableName";
-  public static final String PARTITIONS_TO_PRUNE =
+  private static final String PARTITIONS_TO_PRUNE =
       "mapreduce.input.carboninputformat.partitions.to.prune";
   public static final String UPADTE_T =
       "mapreduce.input.carboninputformat.partitions.to.prune";
@@ -339,7 +338,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
    * @return List<InputSplit> list of CarbonInputSplit
    * @throws IOException
    */
-  @Override public List<InputSplit> getSplits(JobContext job) throws IOException {
+  @Override
+  public List<InputSplit> getSplits(JobContext job) throws IOException {
     AbsoluteTableIdentifier identifier = getAbsoluteTableIdentifier(job.getConfiguration());
     SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
     CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());
@@ -808,28 +808,29 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     return split;
   }
 
-  @Override public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
+  @Override
+  public RecordReader<Void, T> createRecordReader(InputSplit inputSplit,
       TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
     Configuration configuration = taskAttemptContext.getConfiguration();
-    QueryModel queryModel = getQueryModel(inputSplit, taskAttemptContext);
+    QueryModel queryModel = createQueryModel(inputSplit, taskAttemptContext);
     CarbonReadSupport<T> readSupport = getReadSupportClass(configuration);
     return new CarbonRecordReader<T>(queryModel, readSupport);
   }
 
-  public QueryModel getQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+  public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     CarbonTable carbonTable = getOrCreateCarbonTable(configuration);
     TableProvider tableProvider = new SingleTableProvider(carbonTable);
-    // getting the table absoluteTableIdentifier from the carbonTable
-    // to avoid unnecessary deserialization
-    AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
 
     // query plan includes projection column
-    String projection = getColumnProjection(configuration);
-    CarbonQueryPlan queryPlan = CarbonInputFormatUtil.createQueryPlan(carbonTable, projection);
-    QueryModel queryModel = QueryModel.createModel(identifier, queryPlan, carbonTable,
-        getDataTypeConverter(configuration));
+    String projectionString = getColumnProjection(configuration);
+    String[] projectionColumnNames = null;
+    if (projectionString != null) {
+      projectionColumnNames = projectionString.split(",");
+    }
+    QueryModel queryModel = carbonTable.createQueryWithProjection(
+        projectionColumnNames, getDataTypeConverter(configuration));
 
     // set the filter to the query model in order to filter blocklet before scan
     Expression filter = getFilterPredicates(configuration);
@@ -884,7 +885,8 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     return readSupport;
   }
 
-  @Override protected boolean isSplitable(JobContext context, Path filename) {
+  @Override
+  protected boolean isSplitable(JobContext context, Path filename) {
     try {
       // Don't split the file if it is local file system
       FileSystem fileSystem = filename.getFileSystem(context.getConfiguration());
@@ -898,16 +900,6 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
   }
 
   /**
-   * required to be moved to core
-   *
-   * @return updateExtension
-   */
-  private String getUpdateExtension() {
-    // TODO: required to modify when supporting update, mostly will be update timestamp
-    return "update";
-  }
-
-  /**
    * return valid segment to access
    */
   private Segment[] getSegmentsToAccess(JobContext job) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
index a590a5b..0fe0cbf 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/internal/index/impl/InMemoryBTreeIndex.java
@@ -176,9 +176,7 @@ class InMemoryBTreeIndex implements Index {
         filterredBlocks = filterExpressionProcessor.getFilterredBlocks(
             abstractIndex.getDataRefNode(),
             resolver,
-            abstractIndex,
-            identifier
-        );
+            abstractIndex);
       }
       resultFilterredBlocks.addAll(filterredBlocks);
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
index 95a7af0..1e227c4 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamRecordReader.java
@@ -156,7 +156,7 @@ public class CarbonStreamRecordReader extends RecordReader<Void, Object> {
     hadoopConf = context.getConfiguration();
     if (model == null) {
       CarbonTableInputFormat format = new CarbonTableInputFormat<Object>();
-      model = format.getQueryModel(split, context);
+      model = format.createQueryModel(split, context);
     }
     carbonTable = model.getTable();
     List<CarbonDimension> dimensions =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java
index 89a4a9a..2f28861 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/BlockLevelTraverser.java
@@ -67,7 +67,7 @@ public class BlockLevelTraverser {
       blockName = CarbonTablePath.getCarbonDataFileName(blockName);
       blockName = blockName + CarbonTablePath.getCarbonDataExtension();
 
-      long rowCount = currentBlock.nodeSize();
+      long rowCount = currentBlock.numRows();
 
       String key = CarbonUpdateUtil.getSegmentBlockNameKey(segId, blockName);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 056c27b..9f8c5ec 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -26,18 +26,12 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.FilterExpressionProcessor;
 import org.apache.carbondata.core.scan.filter.TableProvider;
 import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer;
-import org.apache.carbondata.core.scan.filter.intf.FilterOptimizerBasic;
 import org.apache.carbondata.core.scan.filter.optimizer.RangeFilterOptmizer;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat;
 
@@ -52,45 +46,14 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
  */
 public class CarbonInputFormatUtil {
 
-  public static CarbonQueryPlan createQueryPlan(CarbonTable carbonTable, String columnString) {
-    String[] columns = null;
-    if (columnString != null) {
-      columns = columnString.split(",");
-    }
-    String factTableName = carbonTable.getTableName();
-    CarbonQueryPlan plan = new CarbonQueryPlan(carbonTable.getDatabaseName(), factTableName);
-    // fill dimensions
-    // If columns are null, set all dimensions and measures
-    int i = 0;
-    if (columns != null) {
-      for (String column : columns) {
-        CarbonDimension dimensionByName = carbonTable.getDimensionByName(factTableName, column);
-        if (dimensionByName != null) {
-          addQueryDimension(plan, i, dimensionByName);
-          i++;
-        } else {
-          CarbonMeasure measure = carbonTable.getMeasureByName(factTableName, column);
-          if (measure == null) {
-            throw new RuntimeException(column + " column not found in the table " + factTableName);
-          }
-          addQueryMeasure(plan, i, measure);
-          i++;
-        }
-      }
-    }
-
-    plan.setQueryId(System.nanoTime() + "");
-    return plan;
-  }
-
   public static <V> CarbonTableInputFormat<V> createCarbonInputFormat(
       AbsoluteTableIdentifier identifier,
       Job job) throws IOException {
     CarbonTableInputFormat<V> carbonInputFormat = new CarbonTableInputFormat<>();
-    carbonInputFormat.setDatabaseName(job.getConfiguration(),
-        identifier.getCarbonTableIdentifier().getDatabaseName());
-    carbonInputFormat
-        .setTableName(job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
+    CarbonTableInputFormat.setDatabaseName(
+        job.getConfiguration(), identifier.getCarbonTableIdentifier().getDatabaseName());
+    CarbonTableInputFormat.setTableName(
+        job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
     FileInputFormat.addInputPath(job, new Path(identifier.getTablePath()));
     return carbonInputFormat;
   }
@@ -98,30 +61,16 @@ public class CarbonInputFormatUtil {
   public static <V> CarbonTableInputFormat<V> createCarbonTableInputFormat(
       AbsoluteTableIdentifier identifier, List<String> partitionId, Job job) throws IOException {
     CarbonTableInputFormat<V> carbonTableInputFormat = new CarbonTableInputFormat<>();
-    carbonTableInputFormat.setPartitionIdList(job.getConfiguration(), partitionId);
-    carbonTableInputFormat.setDatabaseName(job.getConfiguration(),
-        identifier.getCarbonTableIdentifier().getDatabaseName());
-    carbonTableInputFormat
-        .setTableName(job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
+    CarbonTableInputFormat.setPartitionIdList(
+        job.getConfiguration(), partitionId);
+    CarbonTableInputFormat.setDatabaseName(
+        job.getConfiguration(), identifier.getCarbonTableIdentifier().getDatabaseName());
+    CarbonTableInputFormat.setTableName(
+        job.getConfiguration(), identifier.getCarbonTableIdentifier().getTableName());
     FileInputFormat.addInputPath(job, new Path(identifier.getTablePath()));
     return carbonTableInputFormat;
   }
 
-  private static void addQueryMeasure(CarbonQueryPlan plan, int order, CarbonMeasure measure) {
-    QueryMeasure queryMeasure = new QueryMeasure(measure.getColName());
-    queryMeasure.setQueryOrder(order);
-    queryMeasure.setMeasure(measure);
-    plan.addMeasure(queryMeasure);
-  }
-
-  private static void addQueryDimension(CarbonQueryPlan plan, int order,
-      CarbonDimension dimension) {
-    QueryDimension queryDimension = new QueryDimension(dimension.getColName());
-    queryDimension.setQueryOrder(order);
-    queryDimension.setDimension(dimension);
-    plan.addDimension(queryDimension);
-  }
-
   public static void processFilterExpression(Expression filterExpression, CarbonTable carbonTable,
       boolean[] isFilterDimensions, boolean[] isFilterMeasures) {
     QueryModel.processFilterExpression(carbonTable, filterExpression, isFilterDimensions,
@@ -130,7 +79,7 @@ public class CarbonInputFormatUtil {
     if (null != filterExpression) {
       // Optimize Filter Expression and fit RANGE filters is conditions apply.
       FilterOptimizer rangeFilterOptimizer =
-          new RangeFilterOptmizer(new FilterOptimizerBasic(), filterExpression);
+          new RangeFilterOptmizer(filterExpression);
       rangeFilterOptimizer.optimizeFilter();
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
index f109e1c..1b57f93 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
@@ -30,7 +30,6 @@ import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.filter.SingleTableProvider;
 import org.apache.carbondata.core.scan.filter.TableProvider;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
-import org.apache.carbondata.core.scan.model.CarbonQueryPlan;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.util.DataTypeConverterImpl;
 import org.apache.carbondata.hadoop.CarbonInputSplit;
@@ -140,11 +139,11 @@ public class MapredCarbonInputFormat extends CarbonTableInputFormat<ArrayWritabl
 
     AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
 
-    String projection = getProjection(configuration, carbonTable,
+    String projectionString = getProjection(configuration, carbonTable,
         identifier.getCarbonTableIdentifier().getTableName());
-    CarbonQueryPlan queryPlan = CarbonInputFormatUtil.createQueryPlan(carbonTable, projection);
-    QueryModel queryModel =
-        QueryModel.createModel(identifier, queryPlan, carbonTable, new DataTypeConverterImpl());
+    String[] projectionColumns = projectionString.split(",");
+    QueryModel queryModel = carbonTable.createQueryWithProjection(
+        projectionColumns, new DataTypeConverterImpl());
     // set the filter to the query model in order to filter blocklet before scan
     Expression filter = getFilterPredicates(configuration);
     CarbonInputFormatUtil.processFilterExpression(filter, carbonTable, null, null);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
deleted file mode 100644
index 9a8f8c5..0000000
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbonVectorizedRecordReader.java
+++ /dev/null
@@ -1,243 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.presto;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.metadata.datatype.StructField;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.scan.executor.QueryExecutor;
-import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
-import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.iterator.AbstractDetailQueryResultIterator;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.hadoop.AbstractRecordReader;
-import org.apache.carbondata.hadoop.CarbonInputSplit;
-import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
-
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * A specialized RecordReader that reads into InternalRows or ColumnarBatches directly using the
- * carbondata column APIs and fills the data directly into columns.
- */
-class CarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
-
-  private int batchIdx = 0;
-
-  private int numBatched = 0;
-
-  private CarbonVectorBatch columnarBatch;
-
-  private CarbonColumnarBatch carbonColumnarBatch;
-
-  /**
-   * If true, this class returns batches instead of rows.
-   */
-  private boolean returnColumnarBatch;
-
-  private QueryModel queryModel;
-
-  private AbstractDetailQueryResultIterator iterator;
-
-  private QueryExecutor queryExecutor;
-
-  public CarbonVectorizedRecordReader(QueryExecutor queryExecutor, QueryModel queryModel, AbstractDetailQueryResultIterator iterator) {
-    this.queryModel = queryModel;
-    this.iterator = iterator;
-    this.queryExecutor = queryExecutor;
-    enableReturningBatches();
-  }
-
-  /**
-   * Implementation of RecordReader API.
-   */
-  @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
-      throws IOException, InterruptedException, UnsupportedOperationException {
-    // The input split can contain single HDFS block or multiple blocks, so firstly get all the
-    // blocks and then set them in the query model.
-    List<CarbonInputSplit> splitList;
-    if (inputSplit instanceof CarbonInputSplit) {
-      splitList = new ArrayList<>(1);
-      splitList.add((CarbonInputSplit) inputSplit);
-    } else if (inputSplit instanceof CarbonMultiBlockSplit) {
-      // contains multiple blocks, this is an optimization for concurrent query.
-      CarbonMultiBlockSplit multiBlockSplit = (CarbonMultiBlockSplit) inputSplit;
-      splitList = multiBlockSplit.getAllSplits();
-    } else {
-      throw new RuntimeException("unsupported input split type: " + inputSplit);
-    }
-    List<TableBlockInfo> tableBlockInfoList = CarbonInputSplit.createBlocks(splitList);
-    queryModel.setTableBlockInfos(tableBlockInfoList);
-    queryModel.setVectorReader(true);
-    try {
-      queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
-      iterator = (AbstractDetailQueryResultIterator) queryExecutor.execute(queryModel);
-    } catch (QueryExecutionException e) {
-      throw new InterruptedException(e.getMessage());
-    }
-  }
-
-  @Override public void close() throws IOException {
-    logStatistics(rowCount, queryModel.getStatisticsRecorder());
-    if (columnarBatch != null) {
-      columnarBatch = null;
-    }
-    // clear dictionary cache
-    Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
-    if (null != columnToDictionaryMapping) {
-      for (Map.Entry<String, Dictionary> entry : columnToDictionaryMapping.entrySet()) {
-        CarbonUtil.clearDictionaryCache(entry.getValue());
-      }
-    }
-    try {
-      queryExecutor.finish();
-    } catch (QueryExecutionException e) {
-      throw new IOException(e);
-    }
-  }
-
-  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
-    resultBatch();
-
-    if (returnColumnarBatch) return nextBatch();
-
-    if (batchIdx >= numBatched) {
-      if (!nextBatch()) return false;
-    }
-    ++batchIdx;
-    return true;
-  }
-
-  @Override public Object getCurrentValue() throws IOException, InterruptedException {
-    if (returnColumnarBatch) {
-      rowCount += columnarBatch.numValidRows();
-      return columnarBatch;
-    } else {
-      return null;
-    }
-  }
-
-  @Override public Void getCurrentKey() throws IOException, InterruptedException {
-    return null;
-  }
-
-  @Override public float getProgress() throws IOException, InterruptedException {
-    // TODO : Implement it based on total number of rows it is going to retrive.
-    return 0;
-  }
-
-  /**
-   * Returns the ColumnarBatch object that will be used for all rows returned by this reader.
-   * This object is reused. Calling this enables the vectorized reader. This should be called
-   * before any calls to nextKeyValue/nextBatch.
-   */
-
-  private void initBatch() {
-    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
-    List<QueryMeasure> queryMeasures = queryModel.getQueryMeasures();
-    StructField[] fields = new StructField[queryDimension.size() + queryMeasures.size()];
-    for (int i = 0; i < queryDimension.size(); i++) {
-      QueryDimension dim = queryDimension.get(i);
-      if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(dim.getDimension().getDataType());
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-           generator.getReturnType());
-      } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-            dim.getDimension().getDataType());
-      } else if (dim.getDimension().isComplex()) {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-           dim.getDimension().getDataType());
-      } else {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
-            DataTypes.INT);
-      }
-    }
-
-    for (QueryMeasure msr : queryMeasures) {
-      DataType dataType = msr.getMeasure().getDataType();
-      if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT || dataType == DataTypes.INT
-          || dataType == DataTypes.LONG) {
-        fields[msr.getQueryOrder()] =
-            new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
-      } else if (DataTypes.isDecimal(dataType)) {
-        fields[msr.getQueryOrder()] =
-            new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
-      } else {
-        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(), DataTypes.DOUBLE);
-      }
-    }
-
-    columnarBatch = CarbonVectorBatch.allocate(fields);
-    CarbonColumnVector[] vectors = new CarbonColumnVector[fields.length];
-    boolean[] filteredRows = new boolean[columnarBatch.capacity()];
-    for (int i = 0; i < fields.length; i++) {
-      vectors[i] = new CarbonColumnVectorWrapper(columnarBatch.column(i), filteredRows);
-    }
-    carbonColumnarBatch = new CarbonColumnarBatch(vectors, columnarBatch.capacity(), filteredRows);
-  }
-
-
-  private CarbonVectorBatch resultBatch() {
-    if (columnarBatch == null) initBatch();
-    return columnarBatch;
-  }
-
-  /*
-   * Can be called before any rows are returned to enable returning columnar batches directly.
-   */
-  private void enableReturningBatches() {
-    returnColumnarBatch = true;
-  }
-
-  /**
-   * Advances to the next batch of rows. Returns false if there are no more.
-   */
-  private boolean nextBatch() {
-    columnarBatch.reset();
-    carbonColumnarBatch.reset();
-    if (iterator.hasNext()) {
-      iterator.processNextBatch(carbonColumnarBatch);
-      int actualSize = carbonColumnarBatch.getActualSize();
-      columnarBatch.setNumRows(actualSize);
-      numBatched = actualSize;
-      batchIdx = 0;
-      return true;
-    }
-    return false;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
index 1679f29..5f1f90a 100644
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataPageSource.java
@@ -54,7 +54,7 @@ class CarbondataPageSource implements ConnectorPageSource {
   private final List<Type> types;
   private final PageBuilder pageBuilder;
   private boolean closed;
-  private CarbonVectorizedRecordReader vectorReader;
+  private PrestoCarbonVectorizedRecordReader vectorReader;
   private CarbonDictionaryDecodeReadSupport<Object[]> readSupport;
   private long sizeOfData = 0;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
index c614fa9..5772fbf 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordCursor.java
@@ -54,7 +54,7 @@ public class CarbondataRecordCursor implements RecordCursor {
   private CarbondataSplit split;
   private CarbonDictionaryDecodeReadSupport readSupport;
   private Tuple3<DataType, Dictionary, Int>[] dictionary;
-  CarbonVectorizedRecordReader vectorizedRecordReader;
+  PrestoCarbonVectorizedRecordReader vectorizedRecordReader;
 
   private long totalBytes;
   private long nanoStart;
@@ -63,7 +63,7 @@ public class CarbondataRecordCursor implements RecordCursor {
 
 
   public CarbondataRecordCursor(CarbonDictionaryDecodeReadSupport readSupport,
-       CarbonVectorizedRecordReader vectorizedRecordReader,
+       PrestoCarbonVectorizedRecordReader vectorizedRecordReader,
       List<CarbondataColumnHandle> columnHandles,
       CarbondataSplit split) {
     this.vectorizedRecordReader = vectorizedRecordReader;
@@ -194,7 +194,7 @@ public class CarbondataRecordCursor implements RecordCursor {
     //todo  delete cache from readSupport
   }
 
-  public CarbonVectorizedRecordReader getVectorizedRecordReader() {
+  public PrestoCarbonVectorizedRecordReader getVectorizedRecordReader() {
     return vectorizedRecordReader;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
index 0f8fe87..286ff0e 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSet.java
@@ -78,8 +78,8 @@ public class CarbondataRecordSet implements RecordSet {
       readSupport
           .initialize(queryModel.getProjectionColumns(), queryModel.getTable());
       CarbonIterator iterator = queryExecutor.execute(queryModel);
-      CarbonVectorizedRecordReader vectorReader =
-          new CarbonVectorizedRecordReader(queryExecutor, queryModel,
+      PrestoCarbonVectorizedRecordReader vectorReader =
+          new PrestoCarbonVectorizedRecordReader(queryExecutor, queryModel,
               (AbstractDetailQueryResultIterator) iterator);
       return new CarbondataRecordCursor(readSupport, vectorReader, columns, split);
     } catch (QueryExecutionException e) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
index f039daf..5a2f831 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataRecordSetProvider.java
@@ -105,7 +105,7 @@ public class CarbondataRecordSetProvider implements ConnectorRecordSetProvider {
           new TaskAttemptContextImpl(jobConf, new TaskAttemptID("", 1, TaskType.MAP, 0, 0));
       CarbonInputSplit carbonInputSplit =
           CarbonLocalInputSplit.convertSplit(carbondataSplit.getLocalInputSplit());
-      queryModel = carbonTableInputFormat.getQueryModel(carbonInputSplit, hadoopAttemptContext);
+      queryModel = carbonTableInputFormat.createQueryModel(carbonInputSplit, hadoopAttemptContext);
       queryModel.setVectorReader(true);
     } catch (IOException e) {
       throw new RuntimeException("Unable to get the Query Model ", e);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
new file mode 100644
index 0000000..a1907db
--- /dev/null
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/PrestoCarbonVectorizedRecordReader.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.presto;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.scan.executor.QueryExecutor;
+import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
+import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.scan.result.iterator.AbstractDetailQueryResultIterator;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.hadoop.AbstractRecordReader;
+import org.apache.carbondata.hadoop.CarbonInputSplit;
+import org.apache.carbondata.hadoop.CarbonMultiBlockSplit;
+
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * A specialized RecordReader that reads into InternalRows or ColumnarBatches directly using the
+ * carbondata column APIs and fills the data directly into columns.
+ */
+class PrestoCarbonVectorizedRecordReader extends AbstractRecordReader<Object> {
+
+  private int batchIdx = 0;
+
+  private int numBatched = 0;
+
+  private CarbonVectorBatch columnarBatch;
+
+  private CarbonColumnarBatch carbonColumnarBatch;
+
+  /**
+   * If true, this class returns batches instead of rows.
+   */
+  private boolean returnColumnarBatch;
+
+  private QueryModel queryModel;
+
+  private AbstractDetailQueryResultIterator iterator;
+
+  private QueryExecutor queryExecutor;
+
+  public PrestoCarbonVectorizedRecordReader(QueryExecutor queryExecutor, QueryModel queryModel, AbstractDetailQueryResultIterator iterator) {
+    this.queryModel = queryModel;
+    this.iterator = iterator;
+    this.queryExecutor = queryExecutor;
+    enableReturningBatches();
+  }
+
+  /**
+   * Implementation of RecordReader API.
+   */
+  @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+      throws IOException, InterruptedException, UnsupportedOperationException {
+    // The input split can contain single HDFS block or multiple blocks, so firstly get all the
+    // blocks and then set them in the query model.
+    List<CarbonInputSplit> splitList;
+    if (inputSplit instanceof CarbonInputSplit) {
+      splitList = new ArrayList<>(1);
+      splitList.add((CarbonInputSplit) inputSplit);
+    } else if (inputSplit instanceof CarbonMultiBlockSplit) {
+      // contains multiple blocks, this is an optimization for concurrent query.
+      CarbonMultiBlockSplit multiBlockSplit = (CarbonMultiBlockSplit) inputSplit;
+      splitList = multiBlockSplit.getAllSplits();
+    } else {
+      throw new RuntimeException("unsupported input split type: " + inputSplit);
+    }
+    List<TableBlockInfo> tableBlockInfoList = CarbonInputSplit.createBlocks(splitList);
+    queryModel.setTableBlockInfos(tableBlockInfoList);
+    queryModel.setVectorReader(true);
+    try {
+      queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
+      iterator = (AbstractDetailQueryResultIterator) queryExecutor.execute(queryModel);
+    } catch (QueryExecutionException e) {
+      throw new InterruptedException(e.getMessage());
+    }
+  }
+
+  @Override public void close() throws IOException {
+    logStatistics(rowCount, queryModel.getStatisticsRecorder());
+    if (columnarBatch != null) {
+      columnarBatch = null;
+    }
+    // clear dictionary cache
+    Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
+    if (null != columnToDictionaryMapping) {
+      for (Map.Entry<String, Dictionary> entry : columnToDictionaryMapping.entrySet()) {
+        CarbonUtil.clearDictionaryCache(entry.getValue());
+      }
+    }
+    try {
+      queryExecutor.finish();
+    } catch (QueryExecutionException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+    resultBatch();
+
+    if (returnColumnarBatch) return nextBatch();
+
+    if (batchIdx >= numBatched) {
+      if (!nextBatch()) return false;
+    }
+    ++batchIdx;
+    return true;
+  }
+
+  @Override public Object getCurrentValue() throws IOException, InterruptedException {
+    if (returnColumnarBatch) {
+      rowCount += columnarBatch.numValidRows();
+      return columnarBatch;
+    } else {
+      return null;
+    }
+  }
+
+  @Override public Void getCurrentKey() throws IOException, InterruptedException {
+    return null;
+  }
+
+  @Override public float getProgress() throws IOException, InterruptedException {
+    // TODO : Implement it based on total number of rows it is going to retrive.
+    return 0;
+  }
+
+  /**
+   * Returns the ColumnarBatch object that will be used for all rows returned by this reader.
+   * This object is reused. Calling this enables the vectorized reader. This should be called
+   * before any calls to nextKeyValue/nextBatch.
+   */
+
+  private void initBatch() {
+    List<ProjectionDimension> queryDimension = queryModel.getProjectionDimensions();
+    List<ProjectionMeasure> queryMeasures = queryModel.getProjectionMeasures();
+    StructField[] fields = new StructField[queryDimension.size() + queryMeasures.size()];
+    for (int i = 0; i < queryDimension.size(); i++) {
+      ProjectionDimension dim = queryDimension.get(i);
+      if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+        DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
+            .getDirectDictionaryGenerator(dim.getDimension().getDataType());
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
+           generator.getReturnType());
+      } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) {
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
+            dim.getDimension().getDataType());
+      } else if (dim.getDimension().isComplex()) {
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
+           dim.getDimension().getDataType());
+      } else {
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
+            DataTypes.INT);
+      }
+    }
+
+    for (ProjectionMeasure msr : queryMeasures) {
+      DataType dataType = msr.getMeasure().getDataType();
+      if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT || dataType == DataTypes.INT
+          || dataType == DataTypes.LONG) {
+        fields[msr.getOrdinal()] =
+            new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
+      } else if (DataTypes.isDecimal(dataType)) {
+        fields[msr.getOrdinal()] =
+            new StructField(msr.getColumnName(), msr.getMeasure().getDataType());
+      } else {
+        fields[msr.getOrdinal()] = new StructField(msr.getColumnName(), DataTypes.DOUBLE);
+      }
+    }
+
+    columnarBatch = CarbonVectorBatch.allocate(fields);
+    CarbonColumnVector[] vectors = new CarbonColumnVector[fields.length];
+    boolean[] filteredRows = new boolean[columnarBatch.capacity()];
+    for (int i = 0; i < fields.length; i++) {
+      vectors[i] = new CarbonColumnVectorWrapper(columnarBatch.column(i), filteredRows);
+    }
+    carbonColumnarBatch = new CarbonColumnarBatch(vectors, columnarBatch.capacity(), filteredRows);
+  }
+
+
+  private CarbonVectorBatch resultBatch() {
+    if (columnarBatch == null) initBatch();
+    return columnarBatch;
+  }
+
+  /*
+   * Can be called before any rows are returned to enable returning columnar batches directly.
+   */
+  private void enableReturningBatches() {
+    returnColumnarBatch = true;
+  }
+
+  /**
+   * Advances to the next batch of rows. Returns false if there are no more.
+   */
+  private boolean nextBatch() {
+    columnarBatch.reset();
+    carbonColumnarBatch.reset();
+    if (iterator.hasNext()) {
+      iterator.processNextBatch(carbonColumnarBatch);
+      int actualSize = carbonColumnarBatch.getActualSize();
+      columnarBatch.setNumRows(actualSize);
+      numBatched = actualSize;
+      batchIdx = 0;
+      return true;
+    }
+    return false;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index ab3ab5d..3c70619 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -171,7 +171,8 @@ class CarbonMergerRDD[K, V](
         LOGGER.info(s"Restructured block exists: $restructuredBlockExists")
         DataTypeUtil.setDataTypeConverter(new SparkDataTypeConverterImpl)
         exec = new CarbonCompactionExecutor(segmentMapping, segmentProperties,
-          carbonTable, dataFileMetadataSegMapping, restructuredBlockExists)
+          carbonTable, dataFileMetadataSegMapping, restructuredBlockExists,
+          new SparkDataTypeConverterImpl)
 
         // fire a query and get the results.
         var result2: java.util.List[RawResultIterator] = null

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 772f702..97be1fb 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -336,7 +336,7 @@ class CarbonScanRDD(
     TaskMetricsMap.getInstance().registerThreadCallback()
     inputMetricsStats.initBytesReadCallback(context, inputSplit)
     val iterator = if (inputSplit.getAllSplits.size() > 0) {
-      val model = format.getQueryModel(inputSplit, attemptContext)
+      val model = format.createQueryModel(inputSplit, attemptContext)
       // get RecordReader by FileFormat
       val reader: RecordReader[Void, Object] = inputSplit.getFileFormat match {
         case FileFormat.ROW_V1 =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
index 7d42130..432d50a 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/ColumnarVectorWrapper.java
@@ -40,7 +40,7 @@ class ColumnarVectorWrapper implements CarbonColumnVector {
 
   private DataType blockDataType;
 
-  public ColumnarVectorWrapper(ColumnVector columnVector, boolean[] filteredRows) {
+  ColumnarVectorWrapper(ColumnVector columnVector, boolean[] filteredRows) {
     this.columnVector = columnVector;
     this.filteredRows = filteredRows;
     this.dataType = CarbonScalaUtil.convertSparkToCarbonDataType(columnVector.dataType());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
index 5d927df..73da878 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/spark/vectorreader/VectorizedCarbonRecordReader.java
@@ -35,8 +35,8 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.QueryExecutor;
 import org.apache.carbondata.core.scan.executor.QueryExecutorFactory;
 import org.apache.carbondata.core.scan.executor.exception.QueryExecutionException;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 import org.apache.carbondata.core.scan.model.QueryModel;
 import org.apache.carbondata.core.scan.result.iterator.AbstractDetailQueryResultIterator;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
@@ -100,7 +100,8 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
   /**
    * Implementation of RecordReader API.
    */
-  @Override public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+  @Override
+  public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException, UnsupportedOperationException {
     // The input split can contain single HDFS block or multiple blocks, so firstly get all the
     // blocks and then set them in the query model.
@@ -145,7 +146,8 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     }
   }
 
-  @Override public void close() throws IOException {
+  @Override
+  public void close() throws IOException {
     logStatistics(rowCount, queryModel.getStatisticsRecorder());
     if (columnarBatch != null) {
       columnarBatch.close();
@@ -165,10 +167,13 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     }
   }
 
-  @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+  @Override
+  public boolean nextKeyValue() throws IOException, InterruptedException {
     resultBatch();
 
-    if (returnColumnarBatch) return nextBatch();
+    if (returnColumnarBatch) {
+      return nextBatch();
+    }
 
     if (batchIdx >= numBatched) {
       if (!nextBatch()) return false;
@@ -177,7 +182,8 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     return true;
   }
 
-  @Override public Object getCurrentValue() throws IOException, InterruptedException {
+  @Override
+  public Object getCurrentValue() throws IOException, InterruptedException {
     if (returnColumnarBatch) {
       int value = columnarBatch.numValidRows();
       rowCount += value;
@@ -190,11 +196,13 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     return columnarBatch.getRow(batchIdx - 1);
   }
 
-  @Override public Void getCurrentKey() throws IOException, InterruptedException {
+  @Override
+  public Void getCurrentKey() throws IOException, InterruptedException {
     return null;
   }
 
-  @Override public float getProgress() throws IOException, InterruptedException {
+  @Override
+  public float getProgress() throws IOException, InterruptedException {
     // TODO : Implement it based on total number of rows it is going to retrive.
     return 0;
   }
@@ -206,44 +214,44 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
    */
 
   private void initBatch(MemoryMode memMode) {
-    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
-    List<QueryMeasure> queryMeasures = queryModel.getQueryMeasures();
+    List<ProjectionDimension> queryDimension = queryModel.getProjectionDimensions();
+    List<ProjectionMeasure> queryMeasures = queryModel.getProjectionMeasures();
     StructField[] fields = new StructField[queryDimension.size() + queryMeasures.size()];
     for (int i = 0; i < queryDimension.size(); i++) {
-      QueryDimension dim = queryDimension.get(i);
+      ProjectionDimension dim = queryDimension.get(i);
       if (dim.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         DirectDictionaryGenerator generator = DirectDictionaryKeyGeneratorFactory
             .getDirectDictionaryGenerator(dim.getDimension().getDataType());
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(generator.getReturnType()), true, null);
       } else if (!dim.getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(dim.getDimension().getDataType()), true,
             null);
       } else if (dim.getDimension().isComplex()) {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(dim.getDimension().getDataType()), true,
             null);
       } else {
-        fields[dim.getQueryOrder()] = new StructField(dim.getColumnName(),
+        fields[dim.getOrdinal()] = new StructField(dim.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(DataTypes.INT), true, null);
       }
     }
 
     for (int i = 0; i < queryMeasures.size(); i++) {
-      QueryMeasure msr = queryMeasures.get(i);
+      ProjectionMeasure msr = queryMeasures.get(i);
       DataType dataType = msr.getMeasure().getDataType();
       if (dataType == DataTypes.BOOLEAN || dataType == DataTypes.SHORT ||
           dataType == DataTypes.INT || dataType == DataTypes.LONG) {
-        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+        fields[msr.getOrdinal()] = new StructField(msr.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(msr.getMeasure().getDataType()), true,
             null);
       } else if (DataTypes.isDecimal(dataType)) {
-        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+        fields[msr.getOrdinal()] = new StructField(msr.getColumnName(),
             new DecimalType(msr.getMeasure().getPrecision(), msr.getMeasure().getScale()), true,
             null);
       } else {
-        fields[msr.getQueryOrder()] = new StructField(msr.getColumnName(),
+        fields[msr.getOrdinal()] = new StructField(msr.getColumnName(),
             CarbonScalaUtil.convertCarbonToSparkDataType(DataTypes.DOUBLE), true, null);
       }
     }
@@ -261,9 +269,8 @@ class VectorizedCarbonRecordReader extends AbstractRecordReader<Object> {
     initBatch(DEFAULT_MEMORY_MODE);
   }
 
-  private ColumnarBatch resultBatch() {
+  private void resultBatch() {
     if (columnarBatch == null) initBatch();
-    return columnarBatch;
   }
 
   /*


[42/49] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

Posted by ja...@apache.org.
[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

Pick up the no-sort fields in the row and pack them as bytes array and skip parsing them during merge sort to reduce CPU consumption

This closes #1792


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

Branch: refs/heads/carbonstore-rebase4
Commit: 93b2efdd39d21713eb1fdbdda99bd0831c0bf995
Parents: 06a7158
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Thu Feb 8 14:35:14 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:20:26 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/util/NonDictionaryUtil.java |  67 +--
 .../presto/util/CarbonDataStoreCreator.scala    |   1 -
 .../load/DataLoadProcessorStepOnSpark.scala     |   6 +-
 .../loading/row/IntermediateSortTempRow.java    | 117 +++++
 .../loading/sort/SortStepRowHandler.java        | 466 +++++++++++++++++++
 .../loading/sort/SortStepRowUtil.java           | 103 ----
 .../sort/unsafe/UnsafeCarbonRowPage.java        | 331 ++-----------
 .../loading/sort/unsafe/UnsafeSortDataRows.java |  57 +--
 .../unsafe/comparator/UnsafeRowComparator.java  |  95 ++--
 .../UnsafeRowComparatorForNormalDIms.java       |  59 ---
 .../UnsafeRowComparatorForNormalDims.java       |  59 +++
 .../sort/unsafe/holder/SortTempChunkHolder.java |   3 +-
 .../holder/UnsafeFinalMergePageHolder.java      |  19 +-
 .../unsafe/holder/UnsafeInmemoryHolder.java     |  21 +-
 .../holder/UnsafeSortTempFileChunkHolder.java   | 138 ++----
 .../merger/UnsafeIntermediateFileMerger.java    | 118 +----
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  27 +-
 .../merger/CompactionResultSortProcessor.java   |   1 -
 .../sort/sortdata/IntermediateFileMerger.java   |  95 +---
 .../IntermediateSortTempRowComparator.java      |  73 +++
 .../sort/sortdata/NewRowComparator.java         |   5 +-
 .../sortdata/NewRowComparatorForNormalDims.java |   3 +-
 .../processing/sort/sortdata/RowComparator.java |  94 ----
 .../sortdata/RowComparatorForNormalDims.java    |  62 ---
 .../SingleThreadFinalSortFilesMerger.java       |  25 +-
 .../processing/sort/sortdata/SortDataRows.java  |  85 +---
 .../sort/sortdata/SortTempFileChunkHolder.java  | 174 ++-----
 .../sort/sortdata/TableFieldStat.java           | 176 +++++++
 28 files changed, 1186 insertions(+), 1294 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
index d6ecfbc..fca1244 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
@@ -82,18 +82,26 @@ public class NonDictionaryUtil {
   }
 
   /**
-   * Method to get the required Dimension from obj []
+   * Method to get the required dictionary Dimension from obj []
    *
    * @param index
    * @param row
    * @return
    */
-  public static Integer getDimension(int index, Object[] row) {
-
-    Integer[] dimensions = (Integer[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
-
+  public static int getDictDimension(int index, Object[] row) {
+    int[] dimensions = (int[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
     return dimensions[index];
+  }
 
+  /**
+   * Method to get the required non-dictionary & complex from 3-parted row
+   * @param index
+   * @param row
+   * @return
+   */
+  public static byte[] getNoDictOrComplex(int index, Object[] row) {
+    byte[][] nonDictArray = (byte[][]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+    return nonDictArray[index];
   }
 
   /**
@@ -108,60 +116,11 @@ public class NonDictionaryUtil {
     return measures[index];
   }
 
-  public static byte[] getByteArrayForNoDictionaryCols(Object[] row) {
-
-    return (byte[]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-  }
-
   public static void prepareOutObj(Object[] out, int[] dimArray, byte[][] byteBufferArr,
       Object[] measureArray) {
-
     out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray;
     out[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX] = byteBufferArr;
     out[WriteStepRowUtil.MEASURE] = measureArray;
 
   }
-
-  /**
-   * This method will extract the single dimension from the complete high card dims byte[].+     *
-   * The format of the byte [] will be,  Totallength,CompleteStartOffsets,Dat
-   *
-   * @param highCardArr
-   * @param index
-   * @param highCardinalityCount
-   * @param outBuffer
-   */
-  public static void extractSingleHighCardDims(byte[] highCardArr, int index,
-      int highCardinalityCount, ByteBuffer outBuffer) {
-    ByteBuffer buff = null;
-    short secIndex = 0;
-    short firstIndex = 0;
-    int length;
-    // if the requested index is a last one then we need to calculate length
-    // based on byte[] length.
-    if (index == highCardinalityCount - 1) {
-      // need to read 2 bytes(1 short) to determine starting offset and
-      // length can be calculated by array length.
-      buff = ByteBuffer.wrap(highCardArr, (index * 2) + 2, 2);
-    } else {
-      // need to read 4 bytes(2 short) to determine starting offset and
-      // length.
-      buff = ByteBuffer.wrap(highCardArr, (index * 2) + 2, 4);
-    }
-
-    firstIndex = buff.getShort();
-    // if it is a last dimension in high card then this will be last
-    // offset.so calculate length from total length
-    if (index == highCardinalityCount - 1) {
-      secIndex = (short) highCardArr.length;
-    } else {
-      secIndex = buff.getShort();
-    }
-
-    length = secIndex - firstIndex;
-
-    outBuffer.position(firstIndex);
-    outBuffer.limit(outBuffer.position() + length);
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index 1d7c791..7203278 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -383,7 +383,6 @@ object CarbonDataStoreCreator {
       .getInstance.createCache(CacheType.REVERSE_DICTIONARY)
 
     for (i <- set.indices) {
-      //      val dim = getDimension(dims, i).get
       val columnIdentifier: ColumnIdentifier =
         new ColumnIdentifier(dims.get(i).getColumnId, null, null)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
index 5124247..0422239 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
@@ -35,7 +35,7 @@ import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.loading.parser.impl.RowParserImpl
-import org.apache.carbondata.processing.loading.sort.SortStepRowUtil
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler
 import org.apache.carbondata.processing.loading.steps.DataWriterProcessorStepImpl
 import org.apache.carbondata.processing.sort.sortdata.SortParameters
 import org.apache.carbondata.processing.store.{CarbonFactHandler, CarbonFactHandlerFactory}
@@ -206,7 +206,7 @@ object DataLoadProcessorStepOnSpark {
     val model: CarbonLoadModel = modelBroadcast.value.getCopyWithTaskNo(index.toString)
     val conf = DataLoadProcessBuilder.createConfiguration(model)
     val sortParameters = SortParameters.createSortParameters(conf)
-    val sortStepRowUtil = new SortStepRowUtil(sortParameters)
+    val sortStepRowHandler = new SortStepRowHandler(sortParameters)
     TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) =>
       wrapException(e, model)
     }
@@ -216,7 +216,7 @@ object DataLoadProcessorStepOnSpark {
 
       override def next(): CarbonRow = {
         val row =
-          new CarbonRow(sortStepRowUtil.convertRow(rows.next().getData))
+          new CarbonRow(sortStepRowHandler.convertRawRowTo3Parts(rows.next().getData))
         rowCounter.add(1)
         row
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java b/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
new file mode 100644
index 0000000..8d351cf
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.row;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.DataTypeUtil;
+
+/**
+ * During sort procedure, each row will be written to sort temp file in this logic format.
+ * an intermediate sort temp row consists 3 parts:
+ * dictSort, noDictSort, noSortDimsAndMeasures(dictNoSort, noDictNoSort, measure)
+ */
+public class IntermediateSortTempRow {
+  private int[] dictSortDims;
+  private byte[][] noDictSortDims;
+  private byte[] noSortDimsAndMeasures;
+
+  public IntermediateSortTempRow(int[] dictSortDims, byte[][] noDictSortDims,
+      byte[] noSortDimsAndMeasures) {
+    this.dictSortDims = dictSortDims;
+    this.noDictSortDims = noDictSortDims;
+    this.noSortDimsAndMeasures = noSortDimsAndMeasures;
+  }
+
+  public int[] getDictSortDims() {
+    return dictSortDims;
+  }
+
+  public byte[][] getNoDictSortDims() {
+    return noDictSortDims;
+  }
+
+  public byte[] getNoSortDimsAndMeasures() {
+    return noSortDimsAndMeasures;
+  }
+
+  /**
+   * deserialize from bytes array to get the no sort fields
+   * @param outDictNoSort stores the dict & no-sort fields
+   * @param outNoDictNoSort stores the no-dict & no-sort fields, including complex
+   * @param outMeasures stores the measure fields
+   * @param dataTypes data type for the measure
+   */
+  public void unpackNoSortFromBytes(int[] outDictNoSort, byte[][] outNoDictNoSort,
+      Object[] outMeasures, DataType[] dataTypes) {
+    ByteBuffer rowBuffer = ByteBuffer.wrap(noSortDimsAndMeasures);
+    // read dict_no_sort
+    int dictNoSortCnt = outDictNoSort.length;
+    for (int i = 0; i < dictNoSortCnt; i++) {
+      outDictNoSort[i] = rowBuffer.getInt();
+    }
+
+    // read no_dict_no_sort (including complex)
+    int noDictNoSortCnt = outNoDictNoSort.length;
+    for (int i = 0; i < noDictNoSortCnt; i++) {
+      short len = rowBuffer.getShort();
+      byte[] bytes = new byte[len];
+      rowBuffer.get(bytes);
+      outNoDictNoSort[i] = bytes;
+    }
+
+    // read measure
+    int measureCnt = outMeasures.length;
+    DataType tmpDataType;
+    Object tmpContent;
+    for (short idx = 0 ; idx < measureCnt; idx++) {
+      if ((byte) 0 == rowBuffer.get()) {
+        outMeasures[idx] = null;
+        continue;
+      }
+
+      tmpDataType = dataTypes[idx];
+      if (DataTypes.BOOLEAN == tmpDataType) {
+        if ((byte) 1 == rowBuffer.get()) {
+          tmpContent = true;
+        } else {
+          tmpContent = false;
+        }
+      } else if (DataTypes.SHORT == tmpDataType) {
+        tmpContent = rowBuffer.getShort();
+      } else if (DataTypes.INT == tmpDataType) {
+        tmpContent = rowBuffer.getInt();
+      } else if (DataTypes.LONG == tmpDataType) {
+        tmpContent = rowBuffer.getLong();
+      } else if (DataTypes.DOUBLE == tmpDataType) {
+        tmpContent = rowBuffer.getDouble();
+      } else if (DataTypes.isDecimal(tmpDataType)) {
+        short len = rowBuffer.getShort();
+        byte[] decimalBytes = new byte[len];
+        rowBuffer.get(decimalBytes);
+        tmpContent = DataTypeUtil.byteToBigDecimal(decimalBytes);
+      } else {
+        throw new IllegalArgumentException("Unsupported data type: " + tmpDataType);
+      }
+      outMeasures[idx] = tmpContent;
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
new file mode 100644
index 0000000..f31a2b9
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
@@ -0,0 +1,466 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.core.memory.CarbonUnsafe;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
+
+/**
+ * This class is used to convert/write/read row in sort step in carbondata.
+ * It consists the following function:
+ * 1. convert raw row & intermediate sort temp row to 3-parted row
+ * 2. read/write intermediate sort temp row to sort temp file & unsafe memory
+ * 3. write raw row directly to sort temp file & unsafe memory as intermediate sort temp row
+ */
+public class SortStepRowHandler implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private int dictSortDimCnt = 0;
+  private int dictNoSortDimCnt = 0;
+  private int noDictSortDimCnt = 0;
+  private int noDictNoSortDimCnt = 0;
+  private int measureCnt;
+
+  // indices for dict & sort dimension columns
+  private int[] dictSortDimIdx;
+  // indices for dict & no-sort dimension columns
+  private int[] dictNoSortDimIdx;
+  // indices for no-dict & sort dimension columns
+  private int[] noDictSortDimIdx;
+  // indices for no-dict & no-sort dimension columns, including complex columns
+  private int[] noDictNoSortDimIdx;
+  // indices for measure columns
+  private int[] measureIdx;
+
+  private DataType[] dataTypes;
+
+  /**
+   * constructor
+   * @param tableFieldStat table field stat
+   */
+  public SortStepRowHandler(TableFieldStat tableFieldStat) {
+    this.dictSortDimCnt = tableFieldStat.getDictSortDimCnt();
+    this.dictNoSortDimCnt = tableFieldStat.getDictNoSortDimCnt();
+    this.noDictSortDimCnt = tableFieldStat.getNoDictSortDimCnt();
+    this.noDictNoSortDimCnt = tableFieldStat.getNoDictNoSortDimCnt();
+    this.measureCnt = tableFieldStat.getMeasureCnt();
+    this.dictSortDimIdx = tableFieldStat.getDictSortDimIdx();
+    this.dictNoSortDimIdx = tableFieldStat.getDictNoSortDimIdx();
+    this.noDictSortDimIdx = tableFieldStat.getNoDictSortDimIdx();
+    this.noDictNoSortDimIdx = tableFieldStat.getNoDictNoSortDimIdx();
+    this.measureIdx = tableFieldStat.getMeasureIdx();
+    this.dataTypes = tableFieldStat.getMeasureDataType();
+  }
+
+  /**
+   * constructor
+   * @param sortParameters sort parameters
+   */
+  public SortStepRowHandler(SortParameters sortParameters) {
+    this(new TableFieldStat(sortParameters));
+  }
+
+  /**
+   * Convert carbon row from raw format to 3-parted format.
+   * This method is used in global-sort.
+   *
+   * @param row raw row whose length is the same as field number
+   * @return 3-parted row whose length is 3. (1 for dict dims ,1 for non-dict and complex,
+   * 1 for measures)
+   */
+  public Object[] convertRawRowTo3Parts(Object[] row) {
+    Object[] holder = new Object[3];
+    try {
+      int[] dictDims
+          = new int[this.dictSortDimCnt + this.dictNoSortDimCnt];
+      byte[][] nonDictArray = new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt][];
+      Object[] measures = new Object[this.measureCnt];
+
+      // convert dict & data
+      int idxAcc = 0;
+      for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+        dictDims[idxAcc++] = (int) row[this.dictSortDimIdx[idx]];
+      }
+
+      // convert dict & no-sort
+      for (int idx = 0; idx < this.dictNoSortDimCnt; idx++) {
+        dictDims[idxAcc++] = (int) row[this.dictNoSortDimIdx[idx]];
+      }
+      // convert no-dict & sort
+      idxAcc = 0;
+      for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+        nonDictArray[idxAcc++] = (byte[]) row[this.noDictSortDimIdx[idx]];
+      }
+      // convert no-dict & no-sort
+      for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) {
+        nonDictArray[idxAcc++] = (byte[]) row[this.noDictNoSortDimIdx[idx]];
+      }
+
+      // convert measure data
+      for (int idx = 0; idx < this.measureCnt; idx++) {
+        measures[idx] = row[this.measureIdx[idx]];
+      }
+
+      NonDictionaryUtil.prepareOutObj(holder, dictDims, nonDictArray, measures);
+    } catch (Exception e) {
+      throw new RuntimeException("Problem while converting row to 3 parts", e);
+    }
+    return holder;
+  }
+
+  /**
+   * Convert intermediate sort temp row to 3-parted row.
+   * This method is used in the final merge sort to feed rows to the next write step.
+   *
+   * @param sortTempRow intermediate sort temp row
+   * @return 3-parted row
+   */
+  public Object[] convertIntermediateSortTempRowTo3Parted(IntermediateSortTempRow sortTempRow) {
+    int[] dictDims
+        = new int[this.dictSortDimCnt + this.dictNoSortDimCnt];
+    byte[][] noDictArray
+        = new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt][];
+
+    int[] dictNoSortDims = new int[this.dictNoSortDimCnt];
+    byte[][] noDictNoSortDims = new byte[this.noDictNoSortDimCnt][];
+    Object[] measures = new Object[this.measureCnt];
+
+    sortTempRow.unpackNoSortFromBytes(dictNoSortDims, noDictNoSortDims, measures, this.dataTypes);
+
+    // dict dims
+    System.arraycopy(sortTempRow.getDictSortDims(), 0 , dictDims,
+        0, this.dictSortDimCnt);
+    System.arraycopy(dictNoSortDims, 0, dictDims,
+        this.dictSortDimCnt, this.dictNoSortDimCnt);;
+
+    // no dict dims, including complex
+    System.arraycopy(sortTempRow.getNoDictSortDims(), 0,
+        noDictArray, 0, this.noDictSortDimCnt);
+    System.arraycopy(noDictNoSortDims, 0, noDictArray,
+        this.noDictSortDimCnt, this.noDictNoSortDimCnt);
+
+    // measures are already here
+
+    Object[] holder = new Object[3];
+    NonDictionaryUtil.prepareOutObj(holder, dictDims, noDictArray, measures);
+    return holder;
+  }
+
+  /**
+   * Read intermediate sort temp row from InputStream.
+   * This method is used during the merge sort phase to read row from sort temp file.
+   *
+   * @param inputStream input stream
+   * @return a row that contains three parts
+   * @throws IOException if error occrus while reading from stream
+   */
+  public IntermediateSortTempRow readIntermediateSortTempRowFromInputStream(
+      DataInputStream inputStream) throws IOException {
+    int[] dictSortDims = new int[this.dictSortDimCnt];
+    byte[][] noDictSortDims = new byte[this.noDictSortDimCnt][];
+
+    // read dict & sort dim data
+    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+      dictSortDims[idx] = inputStream.readInt();
+    }
+
+    // read no-dict & sort data
+    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+      short len = inputStream.readShort();
+      byte[] bytes = new byte[len];
+      inputStream.readFully(bytes);
+      noDictSortDims[idx] = bytes;
+    }
+
+    // read no-dict dims & measures
+    int len = inputStream.readInt();
+    byte[] noSortDimsAndMeasures = new byte[len];
+    inputStream.readFully(noSortDimsAndMeasures);
+
+    return new IntermediateSortTempRow(dictSortDims, noDictSortDims, noSortDimsAndMeasures);
+  }
+
+  /**
+   * Write intermediate sort temp row to OutputStream
+   * This method is used during the merge sort phase to write row to sort temp file.
+   *
+   * @param sortTempRow intermediate sort temp row
+   * @param outputStream output stream
+   * @throws IOException if error occurs while writing to stream
+   */
+  public void writeIntermediateSortTempRowToOutputStream(IntermediateSortTempRow sortTempRow,
+      DataOutputStream outputStream) throws IOException {
+    // write dict & sort dim
+    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+      outputStream.writeInt(sortTempRow.getDictSortDims()[idx]);
+    }
+
+    // write no-dict & sort dim
+    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+      byte[] bytes = sortTempRow.getNoDictSortDims()[idx];
+      outputStream.writeShort(bytes.length);
+      outputStream.write(bytes);
+    }
+
+    // write packed no-sort dim & measure
+    outputStream.writeInt(sortTempRow.getNoSortDimsAndMeasures().length);
+    outputStream.write(sortTempRow.getNoSortDimsAndMeasures());
+  }
+
+  /**
+   * Write raw row as an intermediate sort temp row to sort temp file.
+   * This method is used in the beginning of the sort phase. Comparing with converting raw row to
+   * intermediate sort temp row and then writing the converted one, Writing raw row directly will
+   * save the intermediate trivial loss.
+   * This method use an array backend buffer to save memory allocation. The buffer will be reused
+   * for all rows (per thread).
+   *
+   * @param row raw row
+   * @param outputStream output stream
+   * @param rowBuffer array backend buffer
+   * @throws IOException if error occurs while writing to stream
+   */
+  public void writeRawRowAsIntermediateSortTempRowToOutputStream(Object[] row,
+      DataOutputStream outputStream, ByteBuffer rowBuffer) throws IOException {
+    // write dict & sort
+    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+      outputStream.writeInt((int) row[this.dictSortDimIdx[idx]]);
+    }
+
+    // write no-dict & sort
+    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+      byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]];
+      outputStream.writeShort(bytes.length);
+      outputStream.write(bytes);
+    }
+
+    // pack no-sort
+    rowBuffer.clear();
+    packNoSortFieldsToBytes(row, rowBuffer);
+    rowBuffer.flip();
+    int packSize = rowBuffer.limit();
+
+    // write no-sort
+    outputStream.writeInt(packSize);
+    outputStream.write(rowBuffer.array(), 0, packSize);
+  }
+
+  /**
+   * Read intermediate sort temp row from unsafe memory.
+   * This method is used during merge sort phase for off-heap sort.
+   *
+   * @param baseObject base object of memory block
+   * @param address address of the row
+   * @return intermediate sort temp row
+   */
+  public IntermediateSortTempRow readIntermediateSortTempRowFromUnsafeMemory(Object baseObject,
+      long address) {
+    int size = 0;
+
+    int[] dictSortDims = new int[this.dictSortDimCnt];
+    byte[][] noDictSortDims = new byte[this.noDictSortDimCnt][];
+
+    // read dict & sort dim
+    for (int idx = 0; idx < dictSortDims.length; idx++) {
+      dictSortDims[idx] = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+      size += 4;
+    }
+
+    // read no-dict & sort dim
+    for (int idx = 0; idx < noDictSortDims.length; idx++) {
+      short length = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+      size += 2;
+      byte[] bytes = new byte[length];
+      CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
+          bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length);
+      size += length;
+      noDictSortDims[idx] = bytes;
+    }
+
+    // read no-sort dims & measures
+    int len = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+    size += 4;
+    byte[] noSortDimsAndMeasures = new byte[len];
+    CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
+        noSortDimsAndMeasures, CarbonUnsafe.BYTE_ARRAY_OFFSET, len);
+
+    return new IntermediateSortTempRow(dictSortDims, noDictSortDims, noSortDimsAndMeasures);
+  }
+
+  /**
+   * Write intermediate sort temp row directly from unsafe memory to stream.
+   * This method is used at the late beginning of the sort phase to write in-memory pages
+   * to sort temp file. Comparing with reading intermediate sort temp row from memory and then
+   * writing it, Writing directly from memory to stream will save the intermediate trivial loss.
+   *
+   * @param baseObject base object of the memory block
+   * @param address base address of the row
+   * @param outputStream output stream
+   * @throws IOException if error occurs while writing to stream
+   */
+  public void writeIntermediateSortTempRowFromUnsafeMemoryToStream(Object baseObject,
+      long address, DataOutputStream outputStream) throws IOException {
+    int size = 0;
+
+    // dict & sort
+    for (int idx = 0; idx < dictSortDimCnt; idx++) {
+      outputStream.writeInt(CarbonUnsafe.getUnsafe().getInt(baseObject, address + size));
+      size += 4;
+    }
+
+    // no-dict & sort
+    for (int idx = 0; idx < noDictSortDimCnt; idx++) {
+      short length = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+      size += 2;
+      byte[] bytes = new byte[length];
+      CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
+          bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length);
+      size += length;
+
+      outputStream.writeShort(length);
+      outputStream.write(bytes);
+    }
+
+    // packed no-sort & measure
+    int len = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+    size += 4;
+    byte[] noSortDimsAndMeasures = new byte[len];
+    CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
+        noSortDimsAndMeasures, CarbonUnsafe.BYTE_ARRAY_OFFSET, len);
+    size += len;
+
+    outputStream.writeInt(len);
+    outputStream.write(noSortDimsAndMeasures);
+  }
+
+  /**
+   * Write raw row as an intermediate sort temp row to memory.
+   * This method is used in the beginning of the off-heap sort phase. Comparing with converting
+   * raw row to intermediate sort temp row and then writing the converted one,
+   * Writing raw row directly will save the intermediate trivial loss.
+   * This method use an array backend buffer to save memory allocation. The buffer will be reused
+   * for all rows (per thread).
+   *
+   * @param row raw row
+   * @param baseObject base object of the memory block
+   * @param address base address for the row
+   * @param rowBuffer array backend buffer
+   * @return number of bytes written to memory
+   */
+  public int writeRawRowAsIntermediateSortTempRowToUnsafeMemory(Object[] row,
+      Object baseObject, long address, ByteBuffer rowBuffer) {
+    int size = 0;
+    // write dict & sort
+    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+      CarbonUnsafe.getUnsafe()
+          .putInt(baseObject, address + size, (int) row[this.dictSortDimIdx[idx]]);
+      size += 4;
+    }
+
+    // write no-dict & sort
+    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+      byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]];
+      CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) bytes.length);
+      size += 2;
+      CarbonUnsafe.getUnsafe()
+          .copyMemory(bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, address + size,
+              bytes.length);
+      size += bytes.length;
+    }
+
+    // convert pack no-sort
+    rowBuffer.clear();
+    packNoSortFieldsToBytes(row, rowBuffer);
+    rowBuffer.flip();
+    int packSize = rowBuffer.limit();
+
+    // write no-sort
+    CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, packSize);
+    size += 4;
+    CarbonUnsafe.getUnsafe()
+        .copyMemory(rowBuffer.array(), CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, address + size,
+            packSize);
+    size += packSize;
+    return size;
+  }
+
+  /**
+   * Pack to no-sort fields to byte array
+   *
+   * @param row raw row
+   * @param rowBuffer byte array backend buffer
+   */
+  private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) {
+    // convert dict & no-sort
+    for (int idx = 0; idx < this.dictNoSortDimCnt; idx++) {
+      rowBuffer.putInt((int) row[this.dictNoSortDimIdx[idx]]);
+    }
+    // convert no-dict & no-sort
+    for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) {
+      byte[] bytes = (byte[]) row[this.noDictNoSortDimIdx[idx]];
+      rowBuffer.putShort((short) bytes.length);
+      rowBuffer.put(bytes);
+    }
+
+    // convert measure
+    Object tmpValue;
+    DataType tmpDataType;
+    for (int idx = 0; idx < this.measureCnt; idx++) {
+      tmpValue = row[this.measureIdx[idx]];
+      tmpDataType = this.dataTypes[idx];
+      if (null == tmpValue) {
+        rowBuffer.put((byte) 0);
+        continue;
+      }
+      rowBuffer.put((byte) 1);
+      if (DataTypes.BOOLEAN == tmpDataType) {
+        if ((boolean) tmpValue) {
+          rowBuffer.put((byte) 1);
+        } else {
+          rowBuffer.put((byte) 0);
+        }
+      } else if (DataTypes.SHORT == tmpDataType) {
+        rowBuffer.putShort((Short) tmpValue);
+      } else if (DataTypes.INT == tmpDataType) {
+        rowBuffer.putInt((Integer) tmpValue);
+      } else if (DataTypes.LONG == tmpDataType) {
+        rowBuffer.putLong((Long) tmpValue);
+      } else if (DataTypes.DOUBLE == tmpDataType) {
+        rowBuffer.putDouble((Double) tmpValue);
+      } else if (DataTypes.isDecimal(tmpDataType)) {
+        byte[] decimalBytes = DataTypeUtil.bigDecimalToByte((BigDecimal) tmpValue);
+        rowBuffer.putShort((short) decimalBytes.length);
+        rowBuffer.put(decimalBytes);
+      } else {
+        throw new IllegalArgumentException("Unsupported data type: " + tmpDataType);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
deleted file mode 100644
index c4e4756..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.sort;
-
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-
-public class SortStepRowUtil {
-  private int measureCount;
-  private int dimensionCount;
-  private int complexDimensionCount;
-  private int noDictionaryCount;
-  private int[] dictDimIdx;
-  private int[] nonDictIdx;
-  private int[] measureIdx;
-
-  public SortStepRowUtil(SortParameters parameters) {
-    this.measureCount = parameters.getMeasureColCount();
-    this.dimensionCount = parameters.getDimColCount();
-    this.complexDimensionCount = parameters.getComplexDimColCount();
-    this.noDictionaryCount = parameters.getNoDictionaryCount();
-    boolean[] isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
-
-    int index = 0;
-    int nonDicIndex = 0;
-    int allCount = 0;
-
-    // be careful that the default value is 0
-    this.dictDimIdx = new int[dimensionCount - noDictionaryCount];
-    this.nonDictIdx = new int[noDictionaryCount + complexDimensionCount];
-    this.measureIdx = new int[measureCount];
-
-    // indices for dict dim columns
-    for (int i = 0; i < isNoDictionaryDimensionColumn.length; i++) {
-      if (isNoDictionaryDimensionColumn[i]) {
-        nonDictIdx[nonDicIndex++] = i;
-      } else {
-        dictDimIdx[index++] = allCount;
-      }
-      allCount++;
-    }
-
-    // indices for non dict dim/complex columns
-    for (int i = 0; i < complexDimensionCount; i++) {
-      nonDictIdx[nonDicIndex++] = allCount;
-      allCount++;
-    }
-
-    // indices for measure columns
-    for (int i = 0; i < measureCount; i++) {
-      measureIdx[i] = allCount;
-      allCount++;
-    }
-  }
-
-  public Object[] convertRow(Object[] data) {
-    // create new row of size 3 (1 for dims , 1 for high card , 1 for measures)
-    Object[] holder = new Object[3];
-    try {
-
-      int[] dictDims = new int[dimensionCount - noDictionaryCount];
-      byte[][] nonDictArray = new byte[noDictionaryCount + complexDimensionCount][];
-      Object[] measures = new Object[measureCount];
-
-      // write dict dim data
-      for (int idx = 0; idx < dictDimIdx.length; idx++) {
-        dictDims[idx] = (int) data[dictDimIdx[idx]];
-      }
-
-      // write non dict dim data
-      for (int idx = 0; idx < nonDictIdx.length; idx++) {
-        nonDictArray[idx] = (byte[]) data[nonDictIdx[idx]];
-      }
-
-      // write measure data
-      for (int idx = 0; idx < measureIdx.length; idx++) {
-        measures[idx] = data[measureIdx[idx]];
-      }
-      NonDictionaryUtil.prepareOutObj(holder, dictDims, nonDictArray, measures);
-
-      // increment number if record read
-    } catch (Exception e) {
-      throw new RuntimeException("Problem while converting row ", e);
-    }
-    //return out row
-    return holder;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
index e5583c2..7ea5cb3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
@@ -19,35 +19,20 @@ package org.apache.carbondata.processing.loading.sort.unsafe;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.Arrays;
+import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.memory.IntPointerBuffer;
 import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.memory.UnsafeSortMemoryManager;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 /**
  * It can keep the data of prescribed size data in offheap/onheap memory and returns it when needed
  */
 public class UnsafeCarbonRowPage {
-
-  private boolean[] noDictionaryDimensionMapping;
-
-  private boolean[] noDictionarySortColumnMapping;
-
-  private int dimensionSize;
-
-  private int measureSize;
-
-  private DataType[] measureDataType;
-
-  private long[] nullSetWords;
-
   private IntPointerBuffer buffer;
 
   private int lastSize;
@@ -62,16 +47,14 @@ public class UnsafeCarbonRowPage {
 
   private long taskId;
 
-  public UnsafeCarbonRowPage(boolean[] noDictionaryDimensionMapping,
-      boolean[] noDictionarySortColumnMapping, int dimensionSize, int measureSize, DataType[] type,
-      MemoryBlock memoryBlock, boolean saveToDisk, long taskId) {
-    this.noDictionaryDimensionMapping = noDictionaryDimensionMapping;
-    this.noDictionarySortColumnMapping = noDictionarySortColumnMapping;
-    this.dimensionSize = dimensionSize;
-    this.measureSize = measureSize;
-    this.measureDataType = type;
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+
+  public UnsafeCarbonRowPage(TableFieldStat tableFieldStat, MemoryBlock memoryBlock,
+      boolean saveToDisk, long taskId) {
+    this.tableFieldStat = tableFieldStat;
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
     this.saveToDisk = saveToDisk;
-    this.nullSetWords = new long[((measureSize - 1) >> 6) + 1];
     this.taskId = taskId;
     buffer = new IntPointerBuffer(this.taskId);
     this.dataBlock = memoryBlock;
@@ -80,255 +63,44 @@ public class UnsafeCarbonRowPage {
     this.managerType = MemoryManagerType.UNSAFE_MEMORY_MANAGER;
   }
 
-  public int addRow(Object[] row) {
-    int size = addRow(row, dataBlock.getBaseOffset() + lastSize);
+  public int addRow(Object[] row, ByteBuffer rowBuffer) {
+    int size = addRow(row, dataBlock.getBaseOffset() + lastSize, rowBuffer);
     buffer.set(lastSize);
     lastSize = lastSize + size;
     return size;
   }
 
-  private int addRow(Object[] row, long address) {
-    if (row == null) {
-      throw new RuntimeException("Row is null ??");
-    }
-    int dimCount = 0;
-    int size = 0;
-    Object baseObject = dataBlock.getBaseObject();
-    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
-      if (noDictionaryDimensionMapping[dimCount]) {
-        byte[] col = (byte[]) row[dimCount];
-        CarbonUnsafe.getUnsafe()
-            .putShort(baseObject, address + size, (short) col.length);
-        size += 2;
-        CarbonUnsafe.getUnsafe().copyMemory(col, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
-            address + size, col.length);
-        size += col.length;
-      } else {
-        int value = (int) row[dimCount];
-        CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, value);
-        size += 4;
-      }
-    }
-
-    // write complex dimensions here.
-    for (; dimCount < dimensionSize; dimCount++) {
-      byte[] col = (byte[]) row[dimCount];
-      CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) col.length);
-      size += 2;
-      CarbonUnsafe.getUnsafe().copyMemory(col, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
-          address + size, col.length);
-      size += col.length;
-    }
-    Arrays.fill(nullSetWords, 0);
-    int nullSetSize = nullSetWords.length * 8;
-    int nullWordLoc = size;
-    size += nullSetSize;
-    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
-      Object value = row[mesCount + dimensionSize];
-      if (null != value) {
-        DataType dataType = measureDataType[mesCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          Boolean bval = (Boolean) value;
-          CarbonUnsafe.getUnsafe().putBoolean(baseObject, address + size, bval);
-          size += 1;
-        } else if (dataType == DataTypes.SHORT) {
-          Short sval = (Short) value;
-          CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, sval);
-          size += 2;
-        } else if (dataType == DataTypes.INT) {
-          Integer ival = (Integer) value;
-          CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, ival);
-          size += 4;
-        } else if (dataType == DataTypes.LONG) {
-          Long val = (Long) value;
-          CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, val);
-          size += 8;
-        } else if (dataType == DataTypes.DOUBLE) {
-          Double doubleVal = (Double) value;
-          CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, doubleVal);
-          size += 8;
-        } else if (DataTypes.isDecimal(dataType)) {
-          BigDecimal decimalVal = (BigDecimal) value;
-          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(decimalVal);
-          CarbonUnsafe.getUnsafe()
-              .putShort(baseObject, address + size, (short) bigDecimalInBytes.length);
-          size += 2;
-          CarbonUnsafe.getUnsafe()
-              .copyMemory(bigDecimalInBytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
-                  address + size, bigDecimalInBytes.length);
-          size += bigDecimalInBytes.length;
-        } else {
-          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
-        }
-        set(nullSetWords, mesCount);
-      } else {
-        unset(nullSetWords, mesCount);
-      }
-    }
-    CarbonUnsafe.getUnsafe().copyMemory(nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET, baseObject,
-        address + nullWordLoc, nullSetSize);
-    return size;
+  /**
+   * add raw row as intermidiate sort temp row to page
+   *
+   * @param row
+   * @param address
+   * @return
+   */
+  private int addRow(Object[] row, long address, ByteBuffer rowBuffer) {
+    return sortStepRowHandler.writeRawRowAsIntermediateSortTempRowToUnsafeMemory(row,
+        dataBlock.getBaseObject(), address, rowBuffer);
   }
 
-  public Object[] getRow(long address, Object[] rowToFill) {
-    int dimCount = 0;
-    int size = 0;
-
-    Object baseObject = dataBlock.getBaseObject();
-    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
-      if (noDictionaryDimensionMapping[dimCount]) {
-        short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-        byte[] col = new byte[aShort];
-        size += 2;
-        CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                col.length);
-        size += col.length;
-        rowToFill[dimCount] = col;
-      } else {
-        int anInt = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-        size += 4;
-        rowToFill[dimCount] = anInt;
-      }
-    }
-
-    // write complex dimensions here.
-    for (; dimCount < dimensionSize; dimCount++) {
-      short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-      byte[] col = new byte[aShort];
-      size += 2;
-      CarbonUnsafe.getUnsafe()
-          .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET, col.length);
-      size += col.length;
-      rowToFill[dimCount] = col;
-    }
-
-    int nullSetSize = nullSetWords.length * 8;
-    Arrays.fill(nullSetWords, 0);
-    CarbonUnsafe.getUnsafe()
-        .copyMemory(baseObject, address + size, nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET,
-            nullSetSize);
-    size += nullSetSize;
-
-    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
-      if (isSet(nullSetWords, mesCount)) {
-        DataType dataType = measureDataType[mesCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          Boolean bval = CarbonUnsafe.getUnsafe().getBoolean(baseObject, address + size);
-          size += 1;
-          rowToFill[dimensionSize + mesCount] = bval;
-        } else if (dataType == DataTypes.SHORT) {
-          Short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-          size += 2;
-          rowToFill[dimensionSize + mesCount] = sval;
-        } else if (dataType == DataTypes.INT) {
-          Integer ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-          size += 4;
-          rowToFill[dimensionSize + mesCount] = ival;
-        } else if (dataType == DataTypes.LONG) {
-          Long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
-          size += 8;
-          rowToFill[dimensionSize + mesCount] = val;
-        } else if (dataType == DataTypes.DOUBLE) {
-          Double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
-          size += 8;
-          rowToFill[dimensionSize + mesCount] = doubleVal;
-        } else if (DataTypes.isDecimal(dataType)) {
-          short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-          byte[] bigDecimalInBytes = new byte[aShort];
-          size += 2;
-          CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
-              CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
-          size += bigDecimalInBytes.length;
-          rowToFill[dimensionSize + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
-        } else {
-          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
-        }
-      } else {
-        rowToFill[dimensionSize + mesCount] = null;
-      }
-    }
-    return rowToFill;
+  /**
+   * get one row from memory address
+   * @param address address
+   * @return one row
+   */
+  public IntermediateSortTempRow getRow(long address) {
+    return sortStepRowHandler.readIntermediateSortTempRowFromUnsafeMemory(
+        dataBlock.getBaseObject(), address);
   }
 
-  public void fillRow(long address, DataOutputStream stream) throws IOException {
-    int dimCount = 0;
-    int size = 0;
-
-    Object baseObject = dataBlock.getBaseObject();
-    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
-      if (noDictionaryDimensionMapping[dimCount]) {
-        short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-        byte[] col = new byte[aShort];
-        size += 2;
-        CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                col.length);
-        size += col.length;
-        stream.writeShort(aShort);
-        stream.write(col);
-      } else {
-        int anInt = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-        size += 4;
-        stream.writeInt(anInt);
-      }
-    }
-
-    // write complex dimensions here.
-    for (; dimCount < dimensionSize; dimCount++) {
-      short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-      byte[] col = new byte[aShort];
-      size += 2;
-      CarbonUnsafe.getUnsafe()
-          .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET, col.length);
-      size += col.length;
-      stream.writeShort(aShort);
-      stream.write(col);
-    }
-
-    int nullSetSize = nullSetWords.length * 8;
-    Arrays.fill(nullSetWords, 0);
-    CarbonUnsafe.getUnsafe()
-        .copyMemory(baseObject, address + size, nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET,
-            nullSetSize);
-    size += nullSetSize;
-    for (int i = 0; i < nullSetWords.length; i++) {
-      stream.writeLong(nullSetWords[i]);
-    }
-
-    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
-      if (isSet(nullSetWords, mesCount)) {
-        DataType dataType = measureDataType[mesCount];
-        if (dataType == DataTypes.SHORT) {
-          short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-          size += 2;
-          stream.writeShort(sval);
-        } else if (dataType == DataTypes.INT) {
-          int ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-          size += 4;
-          stream.writeInt(ival);
-        } else if (dataType == DataTypes.LONG) {
-          long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
-          size += 8;
-          stream.writeLong(val);
-        } else if (dataType == DataTypes.DOUBLE) {
-          double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
-          size += 8;
-          stream.writeDouble(doubleVal);
-        } else if (DataTypes.isDecimal(dataType)) {
-          short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-          byte[] bigDecimalInBytes = new byte[aShort];
-          size += 2;
-          CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
-              CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
-          size += bigDecimalInBytes.length;
-          stream.writeShort(aShort);
-          stream.write(bigDecimalInBytes);
-        } else {
-          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
-        }
-      }
-    }
+  /**
+   * write a row to stream
+   * @param address address of a row
+   * @param stream stream
+   * @throws IOException
+   */
+  public void writeRow(long address, DataOutputStream stream) throws IOException {
+    sortStepRowHandler.writeIntermediateSortTempRowFromUnsafeMemoryToStream(
+        dataBlock.getBaseObject(), address, stream);
   }
 
   public void freeMemory() {
@@ -362,27 +134,8 @@ public class UnsafeCarbonRowPage {
     return dataBlock;
   }
 
-  public static void set(long[] words, int index) {
-    int wordOffset = (index >> 6);
-    words[wordOffset] |= (1L << index);
-  }
-
-  public static void unset(long[] words, int index) {
-    int wordOffset = (index >> 6);
-    words[wordOffset] &= ~(1L << index);
-  }
-
-  public static boolean isSet(long[] words, int index) {
-    int wordOffset = (index >> 6);
-    return ((words[wordOffset] & (1L << index)) != 0);
-  }
-
-  public boolean[] getNoDictionaryDimensionMapping() {
-    return noDictionaryDimensionMapping;
-  }
-
-  public boolean[] getNoDictionarySortColumnMapping() {
-    return noDictionarySortColumnMapping;
+  public TableFieldStat getTableFieldStat() {
+    return tableFieldStat;
   }
 
   public void setNewDataBlock(MemoryBlock newMemoryBlock) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
index 4dd5e44..5d038d3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.processing.loading.sort.unsafe;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -41,13 +42,14 @@ import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparator;
-import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparatorForNormalDIms;
+import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparatorForNormalDims;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeIntermediateMerger;
 import org.apache.carbondata.processing.loading.sort.unsafe.sort.TimSort;
 import org.apache.carbondata.processing.loading.sort.unsafe.sort.UnsafeIntSortDataFormat;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 public class UnsafeSortDataRows {
@@ -69,7 +71,8 @@ public class UnsafeSortDataRows {
    */
 
   private SortParameters parameters;
-
+  private TableFieldStat tableFieldStat;
+  private ThreadLocal<ByteBuffer> rowBuffer;
   private UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger;
 
   private UnsafeCarbonRowPage rowPage;
@@ -94,7 +97,13 @@ public class UnsafeSortDataRows {
   public UnsafeSortDataRows(SortParameters parameters,
       UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger, int inMemoryChunkSize) {
     this.parameters = parameters;
-
+    this.tableFieldStat = new TableFieldStat(parameters);
+    this.rowBuffer = new ThreadLocal<ByteBuffer>() {
+      @Override protected ByteBuffer initialValue() {
+        byte[] backedArray = new byte[2 * 1024 * 1024];
+        return ByteBuffer.wrap(backedArray);
+      }
+    };
     this.unsafeInMemoryIntermediateFileMerger = unsafeInMemoryIntermediateFileMerger;
 
     // observer of writing file in thread
@@ -127,11 +136,7 @@ public class UnsafeSortDataRows {
     if (isMemoryAvailable) {
       UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(baseBlock.size());
     }
-    this.rowPage = new UnsafeCarbonRowPage(parameters.getNoDictionaryDimnesionColumn(),
-        parameters.getNoDictionarySortColumn(),
-        parameters.getDimColCount() + parameters.getComplexDimColCount(),
-        parameters.getMeasureColCount(), parameters.getMeasureDataType(), baseBlock,
-        !isMemoryAvailable, taskId);
+    this.rowPage = new UnsafeCarbonRowPage(tableFieldStat, baseBlock, !isMemoryAvailable, taskId);
     // Delete if any older file exists in sort temp folder
     deleteSortLocationIfExists();
 
@@ -178,7 +183,7 @@ public class UnsafeSortDataRows {
   private void addBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
     for (int i = 0; i < size; i++) {
       if (rowPage.canAdd()) {
-        bytesAdded += rowPage.addRow(rowBatch[i]);
+        bytesAdded += rowPage.addRow(rowBatch[i], rowBuffer.get());
       } else {
         try {
           if (enableInMemoryIntermediateMerge) {
@@ -194,15 +199,8 @@ public class UnsafeSortDataRows {
           if (!saveToDisk) {
             UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(memoryBlock.size());
           }
-          rowPage = new UnsafeCarbonRowPage(
-                  parameters.getNoDictionaryDimnesionColumn(),
-                  parameters.getNoDictionarySortColumn(),
-                  parameters.getDimColCount() + parameters.getComplexDimColCount(),
-                  parameters.getMeasureColCount(),
-                  parameters.getMeasureDataType(),
-                  memoryBlock,
-                  saveToDisk, taskId);
-          bytesAdded += rowPage.addRow(rowBatch[i]);
+          rowPage = new UnsafeCarbonRowPage(tableFieldStat, memoryBlock, saveToDisk, taskId);
+          bytesAdded += rowPage.addRow(rowBatch[i], rowBuffer.get());
         } catch (Exception e) {
           LOGGER.error(
                   "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
@@ -220,7 +218,7 @@ public class UnsafeSortDataRows {
     // if record holder list size is equal to sort buffer size then it will
     // sort the list and then write current list data to file
     if (rowPage.canAdd()) {
-      rowPage.addRow(row);
+      rowPage.addRow(row, rowBuffer.get());
     } else {
       try {
         if (enableInMemoryIntermediateMerge) {
@@ -235,13 +233,8 @@ public class UnsafeSortDataRows {
         if (!saveToDisk) {
           UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(memoryBlock.size());
         }
-        rowPage = new UnsafeCarbonRowPage(
-            parameters.getNoDictionaryDimnesionColumn(),
-            parameters.getNoDictionarySortColumn(),
-            parameters.getDimColCount(), parameters.getMeasureColCount(),
-            parameters.getMeasureDataType(), memoryBlock,
-            saveToDisk, taskId);
-        rowPage.addRow(row);
+        rowPage = new UnsafeCarbonRowPage(tableFieldStat, memoryBlock, saveToDisk, taskId);
+        rowPage.addRow(row, rowBuffer.get());
       } catch (Exception e) {
         LOGGER.error(
             "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
@@ -269,7 +262,7 @@ public class UnsafeSortDataRows {
             new UnsafeRowComparator(rowPage));
       } else {
         timSort.sort(rowPage.getBuffer(), 0, rowPage.getBuffer().getActualSize(),
-            new UnsafeRowComparatorForNormalDIms(rowPage));
+            new UnsafeRowComparatorForNormalDims(rowPage));
       }
       unsafeInMemoryIntermediateFileMerger.addDataChunkToMerge(rowPage);
     } else {
@@ -295,10 +288,9 @@ public class UnsafeSortDataRows {
       // write number of entries to the file
       stream.writeInt(actualSize);
       for (int i = 0; i < actualSize; i++) {
-        rowPage.fillRow(rowPage.getBuffer().get(i) + rowPage.getDataBlock().getBaseOffset(),
-            stream);
+        rowPage.writeRow(
+            rowPage.getBuffer().get(i) + rowPage.getDataBlock().getBaseOffset(), stream);
       }
-
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
     } finally {
@@ -367,7 +359,7 @@ public class UnsafeSortDataRows {
               new UnsafeRowComparator(page));
         } else {
           timSort.sort(page.getBuffer(), 0, page.getBuffer().getActualSize(),
-              new UnsafeRowComparatorForNormalDIms(page));
+              new UnsafeRowComparatorForNormalDims(page));
         }
         if (page.isSaveToDisk()) {
           // create a new file every time
@@ -380,7 +372,8 @@ public class UnsafeSortDataRows {
           writeDataToFile(page, sortTempFile);
           LOGGER.info("Time taken to sort row page with size" + page.getBuffer().getActualSize()
               + " and write is: " + (System.currentTimeMillis() - startTime) + ": location:"
-              + sortTempFile);
+              + sortTempFile + ", sort temp file size in MB is "
+              + sortTempFile.length() * 0.1 * 10 / 1024 / 1024);
           page.freeMemory();
           // add sort temp filename to and arrayList. When the list size reaches 20 then
           // intermediate merging of sort temp files will be triggered

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
index d02be9b..33342dc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
@@ -23,63 +23,25 @@ import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeRowComparator implements Comparator<UnsafeCarbonRow> {
-
-  /**
-   * mapping of dictionary and no dictionary of sort_columns.
-   */
-  private boolean[] noDictionarySortColumnMaping;
-
   private Object baseObject;
+  private TableFieldStat tableFieldStat;
+  private int dictSizeInMemory;
 
   public UnsafeRowComparator(UnsafeCarbonRowPage rowPage) {
-    this.noDictionarySortColumnMaping = rowPage.getNoDictionarySortColumnMapping();
     this.baseObject = rowPage.getDataBlock().getBaseObject();
+    this.tableFieldStat = rowPage.getTableFieldStat();
+    this.dictSizeInMemory = (tableFieldStat.getDictSortDimCnt()
+        + tableFieldStat.getDictNoSortDimCnt()) * 4;
   }
 
   /**
    * Below method will be used to compare two mdkey
    */
   public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
-    int diff = 0;
-    long rowA = rowL.address;
-    long rowB = rowR.address;
-    int sizeA = 0;
-    int sizeB = 0;
-    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
-      if (isNoDictionary) {
-        short aShort1 = CarbonUnsafe.getUnsafe().getShort(baseObject, rowA + sizeA);
-        byte[] byteArr1 = new byte[aShort1];
-        sizeA += 2;
-        CarbonUnsafe.getUnsafe().copyMemory(baseObject, rowA + sizeA, byteArr1,
-            CarbonUnsafe.BYTE_ARRAY_OFFSET, aShort1);
-        sizeA += aShort1;
-
-        short aShort2 = CarbonUnsafe.getUnsafe().getShort(baseObject, rowB + sizeB);
-        byte[] byteArr2 = new byte[aShort2];
-        sizeB += 2;
-        CarbonUnsafe.getUnsafe().copyMemory(baseObject, rowB + sizeB, byteArr2,
-            CarbonUnsafe.BYTE_ARRAY_OFFSET, aShort2);
-        sizeB += aShort2;
-
-        int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
-        if (difference != 0) {
-          return difference;
-        }
-      } else {
-        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
-        sizeA += 4;
-        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
-        sizeB += 4;
-        diff = dimFieldA - dimFieldB;
-        if (diff != 0) {
-          return diff;
-        }
-      }
-    }
-
-    return diff;
+    return compare(rowL, baseObject, rowR, baseObject);
   }
 
   /**
@@ -90,35 +52,40 @@ public class UnsafeRowComparator implements Comparator<UnsafeCarbonRow> {
     int diff = 0;
     long rowA = rowL.address;
     long rowB = rowR.address;
-    int sizeA = 0;
-    int sizeB = 0;
-    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
+    int sizeInDictPartA = 0;
+
+    int sizeInNonDictPartA = 0;
+    int sizeInDictPartB = 0;
+    int sizeInNonDictPartB = 0;
+    for (boolean isNoDictionary : tableFieldStat.getIsSortColNoDictFlags()) {
       if (isNoDictionary) {
-        short aShort1 = CarbonUnsafe.getUnsafe().getShort(baseObjectL, rowA + sizeA);
-        byte[] byteArr1 = new byte[aShort1];
-        sizeA += 2;
+        short lengthA = CarbonUnsafe.getUnsafe().getShort(baseObjectL,
+            rowA + dictSizeInMemory + sizeInNonDictPartA);
+        byte[] byteArr1 = new byte[lengthA];
+        sizeInNonDictPartA += 2;
         CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObjectL, rowA + sizeA, byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                aShort1);
-        sizeA += aShort1;
+            .copyMemory(baseObjectL, rowA + dictSizeInMemory + sizeInNonDictPartA,
+                byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthA);
+        sizeInNonDictPartA += lengthA;
 
-        short aShort2 = CarbonUnsafe.getUnsafe().getShort(baseObjectR, rowB + sizeB);
-        byte[] byteArr2 = new byte[aShort2];
-        sizeB += 2;
+        short lengthB = CarbonUnsafe.getUnsafe().getShort(baseObjectR,
+            rowB + dictSizeInMemory + sizeInNonDictPartB);
+        byte[] byteArr2 = new byte[lengthB];
+        sizeInNonDictPartB += 2;
         CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObjectR, rowB + sizeB, byteArr2, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                aShort2);
-        sizeB += aShort2;
+            .copyMemory(baseObjectR, rowB + dictSizeInMemory + sizeInNonDictPartB,
+                byteArr2, CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthB);
+        sizeInNonDictPartB += lengthB;
 
         int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
         if (difference != 0) {
           return difference;
         }
       } else {
-        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObjectL, rowA + sizeA);
-        sizeA += 4;
-        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObjectR, rowB + sizeB);
-        sizeB += 4;
+        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObjectL, rowA + sizeInDictPartA);
+        sizeInDictPartA += 4;
+        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObjectR, rowB + sizeInDictPartB);
+        sizeInDictPartB += 4;
         diff = dimFieldA - dimFieldB;
         if (diff != 0) {
           return diff;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
deleted file mode 100644
index 483dcb2..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.sort.unsafe.comparator;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.memory.CarbonUnsafe;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
-
-public class UnsafeRowComparatorForNormalDIms implements Comparator<UnsafeCarbonRow> {
-
-  private Object baseObject;
-
-  private int numberOfSortColumns;
-
-  public UnsafeRowComparatorForNormalDIms(UnsafeCarbonRowPage rowPage) {
-    this.baseObject = rowPage.getDataBlock().getBaseObject();
-    this.numberOfSortColumns = rowPage.getNoDictionarySortColumnMapping().length;
-  }
-
-  /**
-   * Below method will be used to compare two mdkey
-   */
-  public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
-    int diff = 0;
-    long rowA = rowL.address;
-    long rowB = rowR.address;
-    int sizeA = 0;
-    int sizeB = 0;
-    for (int i = 0; i < numberOfSortColumns; i++) {
-      int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
-      sizeA += 4;
-      int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
-      sizeB += 4;
-      diff = dimFieldA - dimFieldB;
-      if (diff != 0) {
-        return diff;
-      }
-    }
-
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
new file mode 100644
index 0000000..e9cfb1c
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort.unsafe.comparator;
+
+import java.util.Comparator;
+
+import org.apache.carbondata.core.memory.CarbonUnsafe;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
+
+public class UnsafeRowComparatorForNormalDims implements Comparator<UnsafeCarbonRow> {
+
+  private Object baseObject;
+
+  private int numberOfSortColumns;
+
+  public UnsafeRowComparatorForNormalDims(UnsafeCarbonRowPage rowPage) {
+    this.baseObject = rowPage.getDataBlock().getBaseObject();
+    this.numberOfSortColumns = rowPage.getTableFieldStat().getIsSortColNoDictFlags().length;
+  }
+
+  /**
+   * Below method will be used to compare two mdkey
+   */
+  public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
+    int diff = 0;
+    long rowA = rowL.address;
+    long rowB = rowR.address;
+    int sizeA = 0;
+    int sizeB = 0;
+    for (int i = 0; i < numberOfSortColumns; i++) {
+      int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
+      sizeA += 4;
+      int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
+      sizeB += 4;
+      diff = dimFieldA - dimFieldB;
+      if (diff != 0) {
+        return diff;
+      }
+    }
+
+    return diff;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
index 686e855..d790c41 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 /**
@@ -28,7 +29,7 @@ public interface SortTempChunkHolder extends Comparable<SortTempChunkHolder> {
 
   void readRow()  throws CarbonSortKeyAndGroupByException;
 
-  Object[] getRow();
+  IntermediateSortTempRow getRow();
 
   int numberOfRows();
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
index 6b0cfa6..a776db1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
@@ -19,9 +19,10 @@ package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeInMemoryIntermediateDataMerger;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 
 public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
 
@@ -38,21 +39,18 @@ public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
 
   private UnsafeCarbonRowPage[] rowPages;
 
-  private NewRowComparator comparator;
+  private IntermediateSortTempRowComparator comparator;
 
-  private Object[] currentRow;
-
-  private int columnSize;
+  private IntermediateSortTempRow currentRow;
 
   public UnsafeFinalMergePageHolder(UnsafeInMemoryIntermediateDataMerger merger,
-      boolean[] noDictSortColumnMapping, int columnSize) {
+      boolean[] noDictSortColumnMapping) {
     this.actualSize = merger.getEntryCount();
     this.mergedAddresses = merger.getMergedAddresses();
     this.rowPageIndexes = merger.getRowPageIndexes();
     this.rowPages = merger.getUnsafeCarbonRowPages();
     LOGGER.audit("Processing unsafe inmemory rows page with size : " + actualSize);
-    this.comparator = new NewRowComparator(noDictSortColumnMapping);
-    this.columnSize = columnSize;
+    this.comparator = new IntermediateSortTempRowComparator(noDictSortColumnMapping);
   }
 
   public boolean hasNext() {
@@ -63,12 +61,11 @@ public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
   }
 
   public void readRow() {
-    currentRow = new Object[columnSize];
-    rowPages[rowPageIndexes[counter]].getRow(mergedAddresses[counter], currentRow);
+    currentRow = rowPages[rowPageIndexes[counter]].getRow(mergedAddresses[counter]);
     counter++;
   }
 
-  public Object[] getRow() {
+  public IntermediateSortTempRow getRow() {
     return currentRow;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/93b2efdd/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
index 6f05088..cbcbbae 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
@@ -19,8 +19,9 @@ package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 
 public class UnsafeInmemoryHolder implements SortTempChunkHolder {
 
@@ -33,21 +34,18 @@ public class UnsafeInmemoryHolder implements SortTempChunkHolder {
 
   private UnsafeCarbonRowPage rowPage;
 
-  private Object[] currentRow;
+  private IntermediateSortTempRow currentRow;
 
   private long address;
 
-  private NewRowComparator comparator;
+  private IntermediateSortTempRowComparator comparator;
 
-  private int columnSize;
-
-  public UnsafeInmemoryHolder(UnsafeCarbonRowPage rowPage, int columnSize,
-      int numberOfSortColumns) {
+  public UnsafeInmemoryHolder(UnsafeCarbonRowPage rowPage) {
     this.actualSize = rowPage.getBuffer().getActualSize();
     this.rowPage = rowPage;
     LOGGER.audit("Processing unsafe inmemory rows page with size : " + actualSize);
-    this.comparator = new NewRowComparator(rowPage.getNoDictionarySortColumnMapping());
-    this.columnSize = columnSize;
+    this.comparator = new IntermediateSortTempRowComparator(
+        rowPage.getTableFieldStat().getIsSortColNoDictFlags());
   }
 
   public boolean hasNext() {
@@ -58,13 +56,12 @@ public class UnsafeInmemoryHolder implements SortTempChunkHolder {
   }
 
   public void readRow() {
-    currentRow = new Object[columnSize];
     address = rowPage.getBuffer().get(counter);
-    rowPage.getRow(address + rowPage.getDataBlock().getBaseOffset(), currentRow);
+    currentRow = rowPage.getRow(address + rowPage.getDataBlock().getBaseOffset());
     counter++;
   }
 
-  public Object[] getRow() {
+  public IntermediateSortTempRow getRow() {
     return currentRow;
   }
 


[46/49] carbondata git commit: [CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data loading

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
deleted file mode 100644
index f605b22..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.sort.impl;
-
-import java.io.File;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.metadata.schema.BucketingInfo;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.loading.DataField;
-import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
-import org.apache.carbondata.processing.loading.sort.AbstractMergeSorter;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeSortDataRows;
-import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeIntermediateMerger;
-import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeSingleThreadFinalSortFilesMerger;
-import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * It parallely reads data from array of iterates and do merge sort.
- * First it sorts the data and write to temp files. These temp files will be merge sorted to get
- * final merge sort result.
- * This step is specifically for bucketing, it sorts each bucket data separately and write to
- * temp files.
- */
-public class UnsafeParallelReadMergeSorterWithBucketingImpl extends AbstractMergeSorter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(
-                UnsafeParallelReadMergeSorterWithBucketingImpl.class.getName());
-
-  private SortParameters sortParameters;
-
-  private BucketingInfo bucketingInfo;
-
-  public UnsafeParallelReadMergeSorterWithBucketingImpl(DataField[] inputDataFields,
-      BucketingInfo bucketingInfo) {
-    this.bucketingInfo = bucketingInfo;
-  }
-
-  @Override public void initialize(SortParameters sortParameters) {
-    this.sortParameters = sortParameters;
-  }
-
-  @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
-      throws CarbonDataLoadingException {
-    UnsafeSortDataRows[] sortDataRows = new UnsafeSortDataRows[bucketingInfo.getNumberOfBuckets()];
-    UnsafeIntermediateMerger[] intermediateFileMergers =
-        new UnsafeIntermediateMerger[sortDataRows.length];
-    int inMemoryChunkSizeInMB = CarbonProperties.getInstance().getSortMemoryChunkSizeInMB();
-    inMemoryChunkSizeInMB = inMemoryChunkSizeInMB / bucketingInfo.getNumberOfBuckets();
-    if (inMemoryChunkSizeInMB < 5) {
-      inMemoryChunkSizeInMB = 5;
-    }
-    try {
-      for (int i = 0; i < bucketingInfo.getNumberOfBuckets(); i++) {
-        SortParameters parameters = sortParameters.getCopy();
-        parameters.setPartitionID(i + "");
-        setTempLocation(parameters);
-        intermediateFileMergers[i] = new UnsafeIntermediateMerger(parameters);
-        sortDataRows[i] =
-            new UnsafeSortDataRows(parameters, intermediateFileMergers[i], inMemoryChunkSizeInMB);
-        sortDataRows[i].initialize();
-      }
-    } catch (MemoryException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
-    this.threadStatusObserver = new ThreadStatusObserver(executorService);
-    final int batchSize = CarbonProperties.getInstance().getBatchSize();
-    try {
-      for (int i = 0; i < iterators.length; i++) {
-        executorService.execute(new SortIteratorThread(iterators[i], sortDataRows, this
-            .threadStatusObserver));
-      }
-      executorService.shutdown();
-      executorService.awaitTermination(2, TimeUnit.DAYS);
-      processRowToNextStep(sortDataRows, sortParameters);
-    } catch (Exception e) {
-      checkError();
-      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
-    }
-    checkError();
-    try {
-      for (int i = 0; i < intermediateFileMergers.length; i++) {
-        intermediateFileMergers[i].finish();
-      }
-    } catch (Exception e) {
-      throw new CarbonDataLoadingException(e);
-    }
-
-    Iterator<CarbonRowBatch>[] batchIterator = new Iterator[bucketingInfo.getNumberOfBuckets()];
-    for (int i = 0; i < sortDataRows.length; i++) {
-      batchIterator[i] = new MergedDataIterator(batchSize, intermediateFileMergers[i]);
-    }
-
-    return batchIterator;
-  }
-
-  private UnsafeSingleThreadFinalSortFilesMerger getFinalMerger() {
-    String[] storeLocation = CarbonDataProcessorUtil.getLocalDataFolderLocation(
-        sortParameters.getDatabaseName(), sortParameters.getTableName(),
-        String.valueOf(sortParameters.getTaskNo()), sortParameters.getSegmentId(),
-        false, false);
-    // Set the data file location
-    String[] dataFolderLocation = CarbonDataProcessorUtil.arrayAppend(storeLocation,
-        File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
-    return new UnsafeSingleThreadFinalSortFilesMerger(sortParameters, dataFolderLocation);
-  }
-
-  @Override public void close() {
-  }
-
-  /**
-   * Below method will be used to process data to next step
-   */
-  private boolean processRowToNextStep(UnsafeSortDataRows[] sortDataRows, SortParameters parameters)
-      throws CarbonDataLoadingException {
-    if (null == sortDataRows || sortDataRows.length == 0) {
-      LOGGER.info("Record Processed For table: " + parameters.getTableName());
-      LOGGER.info("Number of Records was Zero");
-      String logMessage = "Summary: Carbon Sort Key Step: Read: " + 0 + ": Write: " + 0;
-      LOGGER.info(logMessage);
-      return false;
-    }
-
-    try {
-      for (int i = 0; i < sortDataRows.length; i++) {
-        // start sorting
-        sortDataRows[i].startSorting();
-      }
-      // check any more rows are present
-      LOGGER.info("Record Processed For table: " + parameters.getTableName());
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordSortRowsStepTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordDictionaryValuesTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
-      return false;
-    } catch (Exception e) {
-      throw new CarbonDataLoadingException(e);
-    }
-  }
-
-  private void setTempLocation(SortParameters parameters) {
-    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(parameters.getDatabaseName(), parameters.getTableName(),
-            parameters.getTaskNo(), parameters.getSegmentId(),
-            false, false);
-    String[] tmpLoc = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath, File.separator,
-        CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
-    parameters.setTempFileLocation(tmpLoc);
-  }
-
-  /**
-   * This thread iterates the iterator and adds the rows to @{@link UnsafeSortDataRows}
-   */
-  private static class SortIteratorThread implements Runnable {
-
-    private Iterator<CarbonRowBatch> iterator;
-
-    private UnsafeSortDataRows[] sortDataRows;
-
-    private ThreadStatusObserver threadStatusObserver;
-
-    public SortIteratorThread(Iterator<CarbonRowBatch> iterator,
-        UnsafeSortDataRows[] sortDataRows, ThreadStatusObserver threadStatusObserver) {
-      this.iterator = iterator;
-      this.sortDataRows = sortDataRows;
-      this.threadStatusObserver = threadStatusObserver;
-    }
-
-    @Override
-    public void run() {
-      try {
-        while (iterator.hasNext()) {
-          CarbonRowBatch batch = iterator.next();
-          int i = 0;
-          while (batch.hasNext()) {
-            CarbonRow row = batch.next();
-            if (row != null) {
-              UnsafeSortDataRows sortDataRow = sortDataRows[row.bucketNumber];
-              synchronized (sortDataRow) {
-                sortDataRow.addRow(row.getData());
-              }
-            }
-          }
-        }
-      } catch (Exception e) {
-        LOGGER.error(e);
-        this.threadStatusObserver.notifyFailed(e);
-      }
-    }
-
-  }
-
-  private class MergedDataIterator extends CarbonIterator<CarbonRowBatch> {
-
-
-    private int batchSize;
-
-    private boolean firstRow;
-
-    private UnsafeIntermediateMerger intermediateMerger;
-
-    public MergedDataIterator(int batchSize,
-        UnsafeIntermediateMerger intermediateMerger) {
-      this.batchSize = batchSize;
-      this.intermediateMerger = intermediateMerger;
-      this.firstRow = true;
-    }
-
-    private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
-
-    @Override public boolean hasNext() {
-      if (firstRow) {
-        firstRow = false;
-        finalMerger = getFinalMerger();
-        List<UnsafeCarbonRowPage> rowPages = intermediateMerger.getRowPages();
-        finalMerger.startFinalMerge(rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]),
-            intermediateMerger.getMergedPages());
-      }
-      return finalMerger.hasNext();
-    }
-
-    @Override public CarbonRowBatch next() {
-      int counter = 0;
-      CarbonRowBatch rowBatch = new CarbonRowBatch(batchSize);
-      while (finalMerger.hasNext() && counter < batchSize) {
-        rowBatch.addRow(new CarbonRow(finalMerger.next()));
-        counter++;
-      }
-      return rowBatch;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithColumnRangeImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithColumnRangeImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithColumnRangeImpl.java
new file mode 100644
index 0000000..99d6627
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithColumnRangeImpl.java
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort.impl;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.metadata.schema.ColumnRangeInfo;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
+import org.apache.carbondata.processing.loading.sort.AbstractMergeSorter;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeSortDataRows;
+import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeIntermediateMerger;
+import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeSingleThreadFinalSortFilesMerger;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * It parallely reads data from array of iterates and do merge sort.
+ * First it sorts the data and write to temp files. These temp files will be merge sorted to get
+ * final merge sort result.
+ * This step is specifically for the data loading with specifying column value range, such as
+ * bucketing, sort_column_bounds, it sorts each range of data separately and write to temp files.
+ */
+public class UnsafeParallelReadMergeSorterWithColumnRangeImpl extends AbstractMergeSorter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(
+          UnsafeParallelReadMergeSorterWithColumnRangeImpl.class.getName());
+
+  private SortParameters originSortParameters;
+  private UnsafeIntermediateMerger[] intermediateFileMergers;
+  private int inMemoryChunkSizeInMB;
+  private AtomicLong rowCounter;
+  private ColumnRangeInfo columnRangeInfo;
+  /**
+   * counters to collect information about rows processed by each range
+   */
+  private List<AtomicLong> insideRowCounterList;
+
+  public UnsafeParallelReadMergeSorterWithColumnRangeImpl(AtomicLong rowCounter,
+      ColumnRangeInfo columnRangeInfo) {
+    this.rowCounter = rowCounter;
+    this.columnRangeInfo = columnRangeInfo;
+  }
+
+  @Override public void initialize(SortParameters sortParameters) {
+    this.originSortParameters = sortParameters;
+    int totalInMemoryChunkSizeInMB = CarbonProperties.getInstance().getSortMemoryChunkSizeInMB();
+    inMemoryChunkSizeInMB = totalInMemoryChunkSizeInMB / columnRangeInfo.getNumOfRanges();
+    if (inMemoryChunkSizeInMB < 5) {
+      inMemoryChunkSizeInMB = 5;
+    }
+    this.insideRowCounterList = new ArrayList<>(columnRangeInfo.getNumOfRanges());
+    for (int i = 0; i < columnRangeInfo.getNumOfRanges(); i++) {
+      insideRowCounterList.add(new AtomicLong(0));
+    }
+  }
+
+  @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
+      throws CarbonDataLoadingException {
+    UnsafeSortDataRows[] sortDataRows = new UnsafeSortDataRows[columnRangeInfo.getNumOfRanges()];
+    intermediateFileMergers = new UnsafeIntermediateMerger[columnRangeInfo.getNumOfRanges()];
+    SortParameters[] sortParameterArray = new SortParameters[columnRangeInfo.getNumOfRanges()];
+    try {
+      for (int i = 0; i < columnRangeInfo.getNumOfRanges(); i++) {
+        SortParameters parameters = originSortParameters.getCopy();
+        parameters.setPartitionID(i + "");
+        parameters.setRangeId(i);
+        sortParameterArray[i] = parameters;
+        setTempLocation(parameters);
+        intermediateFileMergers[i] = new UnsafeIntermediateMerger(parameters);
+        sortDataRows[i] =
+            new UnsafeSortDataRows(parameters, intermediateFileMergers[i], inMemoryChunkSizeInMB);
+        sortDataRows[i].initialize();
+      }
+    } catch (MemoryException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
+    this.threadStatusObserver = new ThreadStatusObserver(executorService);
+    final int batchSize = CarbonProperties.getInstance().getBatchSize();
+    try {
+      for (int i = 0; i < iterators.length; i++) {
+        executorService.execute(new SortIteratorThread(iterators[i], sortDataRows, rowCounter,
+            this.insideRowCounterList, this.threadStatusObserver));
+      }
+      executorService.shutdown();
+      executorService.awaitTermination(2, TimeUnit.DAYS);
+      processRowToNextStep(sortDataRows, originSortParameters);
+    } catch (Exception e) {
+      checkError();
+      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
+    }
+    checkError();
+    try {
+      for (int i = 0; i < intermediateFileMergers.length; i++) {
+        intermediateFileMergers[i].finish();
+      }
+    } catch (Exception e) {
+      throw new CarbonDataLoadingException(e);
+    }
+
+    Iterator<CarbonRowBatch>[] batchIterator = new Iterator[columnRangeInfo.getNumOfRanges()];
+    for (int i = 0; i < sortDataRows.length; i++) {
+      batchIterator[i] =
+          new MergedDataIterator(sortParameterArray[i], batchSize, intermediateFileMergers[i]);
+    }
+
+    return batchIterator;
+  }
+
+  private UnsafeSingleThreadFinalSortFilesMerger getFinalMerger(SortParameters sortParameters) {
+    String[] storeLocation = CarbonDataProcessorUtil
+        .getLocalDataFolderLocation(sortParameters.getDatabaseName(), sortParameters.getTableName(),
+            String.valueOf(sortParameters.getTaskNo()),
+            sortParameters.getSegmentId() + "", false, false);
+    // Set the data file location
+    String[] dataFolderLocation = CarbonDataProcessorUtil.arrayAppend(storeLocation,
+        File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
+    return new UnsafeSingleThreadFinalSortFilesMerger(sortParameters, dataFolderLocation);
+  }
+
+  @Override public void close() {
+    for (int i = 0; i < intermediateFileMergers.length; i++) {
+      intermediateFileMergers[i].close();
+    }
+  }
+
+  /**
+   * Below method will be used to process data to next step
+   */
+  private boolean processRowToNextStep(UnsafeSortDataRows[] sortDataRows, SortParameters parameters)
+      throws CarbonDataLoadingException {
+    if (null == sortDataRows || sortDataRows.length == 0) {
+      LOGGER.info("Record Processed For table: " + parameters.getTableName());
+      LOGGER.info("Number of Records was Zero");
+      String logMessage = "Summary: Carbon Sort Key Step: Read: " + 0 + ": Write: " + 0;
+      LOGGER.info(logMessage);
+      return false;
+    }
+
+    try {
+      for (int i = 0; i < sortDataRows.length; i++) {
+        // start sorting
+        sortDataRows[i].startSorting();
+      }
+      // check any more rows are present
+      LOGGER.info("Record Processed For table: " + parameters.getTableName());
+      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+          .recordSortRowsStepTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
+      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+          .recordDictionaryValuesTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
+      return false;
+    } catch (Exception e) {
+      throw new CarbonDataLoadingException(e);
+    }
+  }
+
+  private void setTempLocation(SortParameters parameters) {
+    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
+        .getLocalDataFolderLocation(parameters.getDatabaseName(), parameters.getTableName(),
+            parameters.getTaskNo(), parameters.getSegmentId(),
+            false, false);
+    String[] tmpLoc = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath, File.separator,
+        CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
+    LOGGER.error("set temp location: " + StringUtils.join(tmpLoc, ", "));
+    parameters.setTempFileLocation(tmpLoc);
+  }
+
+  /**
+   * This thread iterates the iterator and adds the rows to @{@link UnsafeSortDataRows}
+   */
+  private static class SortIteratorThread implements Runnable {
+
+    private Iterator<CarbonRowBatch> iterator;
+
+    private UnsafeSortDataRows[] sortDataRows;
+    private AtomicLong rowCounter;
+    private List<AtomicLong> insideRowCounterList;
+    private ThreadStatusObserver threadStatusObserver;
+
+    public SortIteratorThread(Iterator<CarbonRowBatch> iterator,
+        UnsafeSortDataRows[] sortDataRows, AtomicLong rowCounter,
+        List<AtomicLong> insideRowCounterList,
+        ThreadStatusObserver threadStatusObserver) {
+      this.iterator = iterator;
+      this.sortDataRows = sortDataRows;
+      this.rowCounter = rowCounter;
+      this.insideRowCounterList = insideRowCounterList;
+      this.threadStatusObserver = threadStatusObserver;
+    }
+
+    @Override
+    public void run() {
+      try {
+        while (iterator.hasNext()) {
+          CarbonRowBatch batch = iterator.next();
+          int i = 0;
+          while (batch.hasNext()) {
+            CarbonRow row = batch.next();
+            if (row != null) {
+              UnsafeSortDataRows sortDataRow = sortDataRows[row.getRangeId()];
+              synchronized (sortDataRow) {
+                rowCounter.getAndIncrement();
+                insideRowCounterList.get(row.getRangeId()).getAndIncrement();
+                sortDataRow.addRow(row.getData());
+              }
+            }
+          }
+        }
+        LOGGER.info("Rows processed by each range: " + insideRowCounterList);
+      } catch (Exception e) {
+        LOGGER.error(e);
+        this.threadStatusObserver.notifyFailed(e);
+      }
+    }
+  }
+
+  private class MergedDataIterator extends CarbonIterator<CarbonRowBatch> {
+
+    private SortParameters sortParameters;
+
+    private int batchSize;
+
+    private boolean firstRow;
+
+    private UnsafeIntermediateMerger intermediateMerger;
+
+    public MergedDataIterator(SortParameters sortParameters, int batchSize,
+        UnsafeIntermediateMerger intermediateMerger) {
+      this.sortParameters = sortParameters;
+      this.batchSize = batchSize;
+      this.intermediateMerger = intermediateMerger;
+      this.firstRow = true;
+    }
+
+    private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
+
+    @Override public boolean hasNext() {
+      if (firstRow) {
+        firstRow = false;
+        finalMerger = getFinalMerger(sortParameters);
+        List<UnsafeCarbonRowPage> rowPages = intermediateMerger.getRowPages();
+        finalMerger.startFinalMerge(rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]),
+            intermediateMerger.getMergedPages());
+      }
+      return finalMerger.hasNext();
+    }
+
+    @Override public CarbonRowBatch next() {
+      int counter = 0;
+      CarbonRowBatch rowBatch = new CarbonRowBatch(batchSize);
+      while (finalMerger.hasNext() && counter < batchSize) {
+        rowBatch.addRow(new CarbonRow(finalMerger.next()));
+        counter++;
+      }
+      return rowBatch;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
index 5d038d3..eaa858e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
@@ -366,9 +366,9 @@ public class UnsafeSortDataRows {
           // create a new file and pick a temp directory randomly every time
           String tmpDir = parameters.getTempFileLocation()[
               new Random().nextInt(parameters.getTempFileLocation().length)];
-          File sortTempFile = new File(
-              tmpDir + File.separator + parameters.getTableName()
-                  + System.nanoTime() + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
+          File sortTempFile = new File(tmpDir + File.separator + parameters.getTableName()
+              + '_' + parameters.getRangeId() + '_' + System.nanoTime()
+              + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
           writeDataToFile(page, sortTempFile);
           LOGGER.info("Time taken to sort row page with size" + page.getBuffer().getActualSize()
               + " and write is: " + (System.currentTimeMillis() - startTime) + ": location:"

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateMerger.java
index 0d24e01..104f3f5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateMerger.java
@@ -112,9 +112,9 @@ public class UnsafeIntermediateMerger {
     String[] tempFileLocations = parameters.getTempFileLocation();
     String targetLocation = tempFileLocations[new Random().nextInt(tempFileLocations.length)];
 
-    File file = new File(
-        targetLocation + File.separator + parameters.getTableName() + System
-            .nanoTime() + CarbonCommonConstants.MERGERD_EXTENSION);
+    File file = new File(targetLocation + File.separator + parameters.getTableName()
+        + '_' + parameters.getRangeId() + '_' + System.nanoTime()
+        + CarbonCommonConstants.MERGERD_EXTENSION);
     UnsafeIntermediateFileMerger merger =
         new UnsafeIntermediateFileMerger(parameters, intermediateFiles, file);
     mergerTask.add(executorService.submit(merger));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
index 64f3c25..b1dc156 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
@@ -147,19 +147,20 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   }
 
   private List<File> getFilesToMergeSort() {
+    // this can be partitionId, bucketId or rangeId, let's call it rangeId
+    final int rangeId = parameters.getRangeId();
+
     FileFilter fileFilter = new FileFilter() {
       public boolean accept(File pathname) {
-        return pathname.getName().startsWith(tableName);
+        return pathname.getName().startsWith(tableName + '_' + rangeId);
       }
     };
 
     // get all the merged files
     List<File> files = new ArrayList<File>(tempFileLocation.length);
-    for (String tempLoc : tempFileLocation)
-    {
+    for (String tempLoc : tempFileLocation) {
       File[] subFiles = new File(tempLoc).listFiles(fileFilter);
-      if (null != subFiles && subFiles.length > 0)
-      {
+      if (null != subFiles && subFiles.length > 0) {
         files.addAll(Arrays.asList(subFiles));
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java
index 90a340d..72a8c25 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java
@@ -19,21 +19,33 @@ package org.apache.carbondata.processing.loading.steps;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.metadata.schema.BucketingInfo;
+import org.apache.carbondata.core.metadata.schema.SortColumnRangeInfo;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.loading.BadRecordsLogger;
 import org.apache.carbondata.processing.loading.BadRecordsLoggerProvider;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
+import org.apache.carbondata.processing.loading.converter.FieldConverter;
 import org.apache.carbondata.processing.loading.converter.RowConverter;
 import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl;
+import org.apache.carbondata.processing.loading.partition.Partitioner;
+import org.apache.carbondata.processing.loading.partition.impl.HashPartitionerImpl;
+import org.apache.carbondata.processing.loading.partition.impl.RangePartitionerImpl;
+import org.apache.carbondata.processing.loading.partition.impl.RawRowComparator;
 import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
 import org.apache.carbondata.processing.util.CarbonBadRecordUtil;
 
+import org.apache.commons.lang3.StringUtils;
+
 /**
  * Replace row data fields with dictionary values if column is configured dictionary encoded.
  * And nondictionary columns as well as complex columns will be converted to byte[].
@@ -41,7 +53,10 @@ import org.apache.carbondata.processing.util.CarbonBadRecordUtil;
 public class DataConverterProcessorStepImpl extends AbstractDataLoadProcessorStep {
 
   private List<RowConverter> converters;
+  private Partitioner<CarbonRow> partitioner;
   private BadRecordsLogger badRecordLogger;
+  private boolean isSortColumnRangeEnabled = false;
+  private boolean isBucketColumnEnabled = false;
 
   public DataConverterProcessorStepImpl(CarbonDataLoadConfiguration configuration,
       AbstractDataLoadProcessorStep child) {
@@ -64,6 +79,81 @@ public class DataConverterProcessorStepImpl extends AbstractDataLoadProcessorSte
     configuration.setCardinalityFinder(converter);
     converters.add(converter);
     converter.initialize();
+
+    if (null != configuration.getBucketingInfo()) {
+      this.isBucketColumnEnabled = true;
+      initializeBucketColumnPartitioner();
+    } else if (null != configuration.getSortColumnRangeInfo()) {
+      this.isSortColumnRangeEnabled = true;
+      initializeSortColumnRangesPartitioner();
+    }
+  }
+
+  /**
+   * initialize partitioner for bucket column
+   */
+  private void initializeBucketColumnPartitioner() {
+    List<Integer> indexes = new ArrayList<>();
+    List<ColumnSchema> columnSchemas = new ArrayList<>();
+    DataField[] inputDataFields = getOutput();
+    BucketingInfo bucketingInfo = configuration.getBucketingInfo();
+    for (int i = 0; i < inputDataFields.length; i++) {
+      for (int j = 0; j < bucketingInfo.getListOfColumns().size(); j++) {
+        if (inputDataFields[i].getColumn().getColName()
+            .equals(bucketingInfo.getListOfColumns().get(j).getColumnName())) {
+          indexes.add(i);
+          columnSchemas.add(inputDataFields[i].getColumn().getColumnSchema());
+          break;
+        }
+      }
+    }
+
+    // hash partitioner to dispatch rows by bucket column
+    this.partitioner =
+        new HashPartitionerImpl(indexes, columnSchemas, bucketingInfo.getNumOfRanges());
+  }
+
+
+  /**
+   * initialize partitioner for sort column ranges
+   */
+  private void initializeSortColumnRangesPartitioner() {
+    // convert user specified sort-column ranges
+    SortColumnRangeInfo sortColumnRangeInfo = configuration.getSortColumnRangeInfo();
+    int rangeValueCnt = sortColumnRangeInfo.getUserSpecifiedRanges().length;
+    CarbonRow[] convertedSortColumnRanges = new CarbonRow[rangeValueCnt];
+    for (int i = 0; i < rangeValueCnt; i++) {
+      Object[] fakeOriginRow = new Object[configuration.getDataFields().length];
+      String[] oneBound = StringUtils.splitPreserveAllTokens(
+          sortColumnRangeInfo.getUserSpecifiedRanges()[i], sortColumnRangeInfo.getSeparator(), -1);
+      // set the corresponding sort column
+      int j = 0;
+      for (int colIdx : sortColumnRangeInfo.getSortColumnIndex()) {
+        fakeOriginRow[colIdx] = oneBound[j++];
+      }
+      CarbonRow fakeCarbonRow = new CarbonRow(fakeOriginRow);
+      convertFakeRow(fakeCarbonRow, sortColumnRangeInfo);
+      convertedSortColumnRanges[i] = fakeCarbonRow;
+    }
+    // sort the range bounds (sort in carbon is a little different from what we think)
+    Arrays.sort(convertedSortColumnRanges,
+        new RawRowComparator(sortColumnRangeInfo.getSortColumnIndex(),
+            sortColumnRangeInfo.getIsSortColumnNoDict()));
+
+    // range partitioner to dispatch rows by sort columns
+    this.partitioner = new RangePartitionerImpl(convertedSortColumnRanges,
+        new RawRowComparator(sortColumnRangeInfo.getSortColumnIndex(),
+            sortColumnRangeInfo.getIsSortColumnNoDict()));
+  }
+
+  // only convert sort column fields
+  private void convertFakeRow(CarbonRow fakeRow, SortColumnRangeInfo sortColumnRangeInfo) {
+    FieldConverter[] fieldConverters = converters.get(0).getFieldConverters();
+    BadRecordLogHolder logHolder = new BadRecordLogHolder();
+    logHolder.setLogged(false);
+    for (int colIdx : sortColumnRangeInfo.getSortColumnIndex()) {
+      fieldConverters[colIdx].convert(fakeRow, logHolder);
+    }
   }
 
   /**
@@ -102,6 +192,10 @@ public class DataConverterProcessorStepImpl extends AbstractDataLoadProcessorSte
   protected CarbonRowBatch processRowBatch(CarbonRowBatch rowBatch, RowConverter localConverter) {
     while (rowBatch.hasNext()) {
       CarbonRow convertRow = localConverter.convert(rowBatch.next());
+      if (isSortColumnRangeEnabled || isBucketColumnEnabled) {
+        short rangeNumber = (short) partitioner.getPartition(convertRow);
+        convertRow.setRangeId(rangeNumber);
+      }
       rowBatch.setPreviousRow(convertRow);
     }
     rowCounter.getAndAdd(rowBatch.getSize());
@@ -134,6 +228,12 @@ public class DataConverterProcessorStepImpl extends AbstractDataLoadProcessorSte
   }
 
   @Override protected String getStepName() {
-    return "Data Converter";
+    if (isBucketColumnEnabled) {
+      return "Data Converter with Bucketing";
+    } else if (isSortColumnRangeEnabled) {
+      return "Data Converter with sort column range";
+    } else {
+      return "Data Converter";
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorWithBucketingStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorWithBucketingStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorWithBucketingStepImpl.java
deleted file mode 100644
index a1181c9..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorWithBucketingStepImpl.java
+++ /dev/null
@@ -1,161 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.steps;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.metadata.schema.BucketingInfo;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
-import org.apache.carbondata.processing.loading.BadRecordsLogger;
-import org.apache.carbondata.processing.loading.BadRecordsLoggerProvider;
-import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.loading.DataField;
-import org.apache.carbondata.processing.loading.converter.RowConverter;
-import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl;
-import org.apache.carbondata.processing.loading.partition.Partitioner;
-import org.apache.carbondata.processing.loading.partition.impl.HashPartitionerImpl;
-import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
-import org.apache.carbondata.processing.util.CarbonBadRecordUtil;
-
-/**
- * Replace row data fields with dictionary values if column is configured dictionary encoded.
- * And nondictionary columns as well as complex columns will be converted to byte[].
- */
-public class DataConverterProcessorWithBucketingStepImpl extends AbstractDataLoadProcessorStep {
-
-  private List<RowConverter> converters;
-
-  private Partitioner<Object[]> partitioner;
-
-  private BadRecordsLogger badRecordLogger;
-
-  public DataConverterProcessorWithBucketingStepImpl(CarbonDataLoadConfiguration configuration,
-      AbstractDataLoadProcessorStep child) {
-    super(configuration, child);
-  }
-
-  @Override
-  public DataField[] getOutput() {
-    return child.getOutput();
-  }
-
-  @Override
-  public void initialize() throws IOException {
-    super.initialize();
-    child.initialize();
-    converters = new ArrayList<>();
-    badRecordLogger = BadRecordsLoggerProvider.createBadRecordLogger(configuration);
-    RowConverter converter =
-        new RowConverterImpl(child.getOutput(), configuration, badRecordLogger);
-    configuration.setCardinalityFinder(converter);
-    converters.add(converter);
-    converter.initialize();
-    List<Integer> indexes = new ArrayList<>();
-    List<ColumnSchema> columnSchemas = new ArrayList<>();
-    DataField[] inputDataFields = getOutput();
-    BucketingInfo bucketingInfo = configuration.getBucketingInfo();
-    for (int i = 0; i < inputDataFields.length; i++) {
-      for (int j = 0; j < bucketingInfo.getListOfColumns().size(); j++) {
-        if (inputDataFields[i].getColumn().getColName()
-            .equals(bucketingInfo.getListOfColumns().get(j).getColumnName())) {
-          indexes.add(i);
-          columnSchemas.add(inputDataFields[i].getColumn().getColumnSchema());
-          break;
-        }
-      }
-    }
-    partitioner =
-        new HashPartitionerImpl(indexes, columnSchemas, bucketingInfo.getNumberOfBuckets());
-  }
-
-  /**
-   * Create the iterator using child iterator.
-   *
-   * @param childIter
-   * @return new iterator with step specific processing.
-   */
-  @Override
-  protected Iterator<CarbonRowBatch> getIterator(final Iterator<CarbonRowBatch> childIter) {
-    return new CarbonIterator<CarbonRowBatch>() {
-      RowConverter localConverter;
-      private boolean first = true;
-      @Override public boolean hasNext() {
-        if (first) {
-          first = false;
-          localConverter = converters.get(0).createCopyForNewThread();
-          converters.add(localConverter);
-        }
-        return childIter.hasNext();
-      }
-
-      @Override public CarbonRowBatch next() {
-        return processRowBatch(childIter.next(), localConverter);
-      }
-    };
-  }
-
-  /**
-   * Process the batch of rows as per the step logic.
-   *
-   * @param rowBatch
-   * @return processed row.
-   */
-  protected CarbonRowBatch processRowBatch(CarbonRowBatch rowBatch, RowConverter localConverter) {
-    while (rowBatch.hasNext()) {
-      CarbonRow row = rowBatch.next();
-      short bucketNumber = (short) partitioner.getPartition(row.getData());
-      CarbonRow convertRow = localConverter.convert(row);
-      convertRow.bucketNumber = bucketNumber;
-      rowBatch.setPreviousRow(convertRow);
-    }
-    rowCounter.getAndAdd(rowBatch.getSize());
-    // reuse the origin batch
-    rowBatch.rewind();
-    return rowBatch;
-  }
-
-  @Override
-  protected CarbonRow processRow(CarbonRow row) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void close() {
-    if (!closed) {
-      super.close();
-      if (null != badRecordLogger) {
-        badRecordLogger.closeStreams();
-        CarbonBadRecordUtil.renameBadRecord(configuration);
-      }
-      if (converters != null) {
-        for (RowConverter converter : converters) {
-          converter.finish();
-        }
-      }
-    }
-  }
-  @Override protected String getStepName() {
-    return "Data Converter with Bucketing";
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
index 58009af..0467b11 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
@@ -17,7 +17,15 @@
 package org.apache.carbondata.processing.loading.steps;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.Callable;
+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.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -25,6 +33,7 @@ import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
@@ -92,7 +101,11 @@ public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
       CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
           .recordDictionaryValue2MdkAdd2FileTime(CarbonTablePath.DEPRECATED_PATITION_ID,
               System.currentTimeMillis());
+      ExecutorService rangeExecutorService = Executors.newFixedThreadPool(iterators.length,
+          new CarbonThreadFactory("WriterForwardPool: " + tableName));
+      List<Future<Void>> rangeExecutorServiceSubmitList = new ArrayList<>(iterators.length);
       int i = 0;
+      // do this concurrently
       for (Iterator<CarbonRowBatch> iterator : iterators) {
         String[] storeLocation = getStoreLocation(tableIdentifier);
 
@@ -112,9 +125,19 @@ public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
         if (!rowsNotExist) {
           finish(dataHandler);
         }
+        rangeExecutorServiceSubmitList.add(
+            rangeExecutorService.submit(new WriterForwarder(iterator, tableIdentifier, i)));
         i++;
       }
-
+      try {
+        rangeExecutorService.shutdown();
+        rangeExecutorService.awaitTermination(2, TimeUnit.DAYS);
+        for (int j = 0; j < rangeExecutorServiceSubmitList.size(); j++) {
+          rangeExecutorServiceSubmitList.get(j).get();
+        }
+      } catch (InterruptedException | ExecutionException e) {
+        throw new CarbonDataWriterException(e);
+      }
     } catch (CarbonDataWriterException e) {
       LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
       throw new CarbonDataLoadingException(
@@ -130,6 +153,51 @@ public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
     return "Data Writer";
   }
 
+  /**
+   * Used to forward rows to different ranges based on range id.
+   */
+  private final class WriterForwarder implements Callable<Void> {
+    private Iterator<CarbonRowBatch> insideRangeIterator;
+    private CarbonTableIdentifier tableIdentifier;
+    private int rangeId;
+
+    public WriterForwarder(Iterator<CarbonRowBatch> insideRangeIterator,
+        CarbonTableIdentifier tableIdentifier, int rangeId) {
+      this.insideRangeIterator = insideRangeIterator;
+      this.tableIdentifier = tableIdentifier;
+      this.rangeId = rangeId;
+    }
+
+    @Override public Void call() throws Exception {
+      LOGGER.info("Process writer forward for table " + tableIdentifier.getTableName()
+          + ", range: " + rangeId);
+      processRange(insideRangeIterator, tableIdentifier, rangeId);
+      return null;
+    }
+  }
+
+  private void processRange(Iterator<CarbonRowBatch> insideRangeIterator,
+      CarbonTableIdentifier tableIdentifier, int rangeId) {
+    String[] storeLocation = getStoreLocation(tableIdentifier);
+
+    CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
+        .createCarbonFactDataHandlerModel(configuration, storeLocation, rangeId, 0);
+    CarbonFactHandler dataHandler = null;
+    boolean rowsNotExist = true;
+    while (insideRangeIterator.hasNext()) {
+      if (rowsNotExist) {
+        rowsNotExist = false;
+        dataHandler = CarbonFactHandlerFactory
+            .createCarbonFactHandler(model, CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
+        dataHandler.initialise();
+      }
+      processBatch(insideRangeIterator.next(), dataHandler);
+    }
+    if (!rowsNotExist) {
+      finish(dataHandler);
+    }
+  }
+
   public void finish(CarbonFactHandler dataHandler) {
     CarbonTableIdentifier tableIdentifier =
         configuration.getTableIdentifier().getCarbonTableIdentifier();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
index a4ac0ea..1a839a2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
@@ -119,9 +119,10 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
   }
 
   private List<File> getFilesToMergeSort() {
+    final int rangeId = sortParameters.getRangeId();
     FileFilter fileFilter = new FileFilter() {
       public boolean accept(File pathname) {
-        return pathname.getName().startsWith(tableName);
+        return pathname.getName().startsWith(tableName + '_' + rangeId);
       }
     };
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
index c7efbd9..a5caf7b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
@@ -212,9 +212,9 @@ public class SortDataRows {
       // create new file and choose folder randomly
       String[] tmpLocation = parameters.getTempFileLocation();
       String locationChosen = tmpLocation[new Random().nextInt(tmpLocation.length)];
-      File file = new File(
-          locationChosen + File.separator + parameters.getTableName() +
-              System.nanoTime() + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
+      File file = new File(locationChosen + File.separator + parameters.getTableName()
+          + '_' + parameters.getRangeId() + '_' + System.nanoTime()
+          + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
       writeDataToFile(recordHolderList, this.entryCount, file);
     }
 
@@ -325,8 +325,9 @@ public class SortDataRows {
         String[] tmpFileLocation = parameters.getTempFileLocation();
         String locationChosen = tmpFileLocation[new Random().nextInt(tmpFileLocation.length)];
         File sortTempFile = new File(
-            locationChosen + File.separator + parameters.getTableName() + System
-                .nanoTime() + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
+            locationChosen + File.separator + parameters.getTableName()
+                + '_' + parameters.getRangeId() + '_' + System.nanoTime()
+                + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
         writeDataToFile(recordHolderArray, recordHolderArray.length, sortTempFile);
         // add sort temp filename to and arrayList. When the list size reaches 20 then
         // intermediate merging of sort temp files will be triggered

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortIntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortIntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortIntermediateFileMerger.java
index 9c995a5..0e3f6bd 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortIntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortIntermediateFileMerger.java
@@ -94,9 +94,9 @@ public class SortIntermediateFileMerger {
   private void startIntermediateMerging(File[] intermediateFiles) {
     int index = new Random().nextInt(parameters.getTempFileLocation().length);
     String chosenTempDir = parameters.getTempFileLocation()[index];
-    File file = new File(
-        chosenTempDir + File.separator + parameters.getTableName() + System
-            .nanoTime() + CarbonCommonConstants.MERGERD_EXTENSION);
+    File file = new File(chosenTempDir + File.separator + parameters.getTableName()
+        + '_' + parameters.getRangeId() + '_' + System.nanoTime()
+        + CarbonCommonConstants.MERGERD_EXTENSION);
     IntermediateFileMerger merger = new IntermediateFileMerger(parameters, intermediateFiles, file);
     mergerTask.add(executorService.submit(merger));
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
index 4d31f87..4d333ed 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
@@ -119,6 +119,7 @@ public class SortParameters implements Serializable {
   private int numberOfCores;
 
   private int batchSortSizeinMb;
+  private int rangeId = 0;
 
   public SortParameters getCopy() {
     SortParameters parameters = new SortParameters();
@@ -147,6 +148,7 @@ public class SortParameters implements Serializable {
     parameters.numberOfNoDictSortColumns = numberOfNoDictSortColumns;
     parameters.numberOfCores = numberOfCores;
     parameters.batchSortSizeinMb = batchSortSizeinMb;
+    parameters.rangeId = rangeId;
     return parameters;
   }
 
@@ -429,6 +431,14 @@ public class SortParameters implements Serializable {
     return parameters;
   }
 
+  public int getRangeId() {
+    return rangeId;
+  }
+
+  public void setRangeId(int rangeId) {
+    this.rangeId = rangeId;
+  }
+
   /**
    * this method will set the boolean mapping for no dictionary sort columns
    *

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index b795696..c0acadd 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -167,12 +167,14 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
 
     blockletProcessingCount = new AtomicInteger(0);
     producerExecutorService = Executors.newFixedThreadPool(numberOfCores,
-        new CarbonThreadFactory("ProducerPool:" + model.getTableName()));
+        new CarbonThreadFactory("ProducerPool:" + model.getTableName()
+            + ", range: " + model.getBucketId()));
     producerExecutorServiceTaskList =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     LOGGER.info("Initializing writer executors");
     consumerExecutorService = Executors
-        .newFixedThreadPool(1, new CarbonThreadFactory("ConsumerPool:" + model.getTableName()));
+        .newFixedThreadPool(1, new CarbonThreadFactory("ConsumerPool:" + model.getTableName()
+            + ", range: " + model.getBucketId()));
     consumerExecutorServiceTaskList = new ArrayList<>(1);
     semaphore = new Semaphore(numberOfCores);
     tablePageList = new TablePageList();


[47/49] carbondata git commit: [CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data loading

Posted by ja...@apache.org.
[CARBONDATA-2091][DataLoad] Support specifying sort column bounds in data loading

Enhance data loading performance by specifying sort column bounds
1. Add row range number during convert-process-step
2. Dispatch rows to each sorter by range number
3. Sort/Write process step can be done concurrently in each range
4. Since all sorttemp files will be written in one folders, we add range
number to the file name to distingush them

Tests added and docs updated

This closes #1953


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

Branch: refs/heads/carbonstore-rebase4
Commit: fa1a100e23c02f98a69591173b4a7b3a4b53eef6
Parents: e059167
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Tue Feb 13 10:58:06 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:22:40 2018 +0800

----------------------------------------------------------------------
 .../constants/CarbonLoadOptionConstants.java    |  10 +
 .../core/datastore/row/CarbonRow.java           |  10 +-
 .../ThriftWrapperSchemaConverterImpl.java       |   2 +-
 .../core/metadata/schema/BucketingInfo.java     |  24 +-
 .../core/metadata/schema/ColumnRangeInfo.java   |  29 ++
 .../metadata/schema/SortColumnRangeInfo.java    |  83 +++++
 docs/data-management-on-carbondata.md           |  11 +
 .../TestLoadDataWithSortColumnBounds.scala      | 348 +++++++++++++++++++
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   2 +-
 .../carbondata/spark/rdd/PartitionDropper.scala |   2 +-
 .../spark/rdd/PartitionSplitter.scala           |   2 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   3 +-
 .../strategy/CarbonLateDecodeStrategy.scala     |   2 +-
 .../loading/CarbonDataLoadConfiguration.java    |  11 +
 .../loading/DataLoadProcessBuilder.java         |  77 +++-
 .../loading/converter/RowConverter.java         |   2 +-
 .../converter/impl/RowConverterImpl.java        |   5 +
 .../loading/model/CarbonLoadModel.java          |  14 +
 .../loading/model/CarbonLoadModelBuilder.java   |   1 +
 .../processing/loading/model/LoadOption.java    |   1 +
 .../partition/impl/HashPartitionerImpl.java     |  10 +-
 .../partition/impl/RangePartitionerImpl.java    |  71 ++++
 .../partition/impl/RawRowComparator.java        |  63 ++++
 .../processing/loading/sort/SorterFactory.java  |  16 +-
 ...arallelReadMergeSorterWithBucketingImpl.java | 272 ---------------
 ...allelReadMergeSorterWithColumnRangeImpl.java | 289 +++++++++++++++
 ...arallelReadMergeSorterWithBucketingImpl.java | 263 --------------
 ...allelReadMergeSorterWithColumnRangeImpl.java | 293 ++++++++++++++++
 .../loading/sort/unsafe/UnsafeSortDataRows.java |   6 +-
 .../unsafe/merger/UnsafeIntermediateMerger.java |   6 +-
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  11 +-
 .../steps/DataConverterProcessorStepImpl.java   | 102 +++++-
 ...ConverterProcessorWithBucketingStepImpl.java | 161 ---------
 .../steps/DataWriterProcessorStepImpl.java      |  70 +++-
 .../SingleThreadFinalSortFilesMerger.java       |   3 +-
 .../processing/sort/sortdata/SortDataRows.java  |  11 +-
 .../sortdata/SortIntermediateFileMerger.java    |   6 +-
 .../sort/sortdata/SortParameters.java           |  10 +
 .../store/CarbonFactDataHandlerColumnar.java    |   6 +-
 39 files changed, 1558 insertions(+), 750 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index a6bf60f..8ff8dc4 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -124,4 +124,14 @@ public final class CarbonLoadOptionConstants {
   public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
       = "carbon.load.skewedDataOptimization.enabled";
   public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
+
+  /**
+   * field delimiter for each field in one bound
+   */
+  public static final String SORT_COLUMN_BOUNDS_FIELD_DELIMITER = ",";
+
+  /**
+   * row delimiter for each sort column bounds
+   */
+  public static final String SORT_COLUMN_BOUNDS_ROW_DELIMITER = ";";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java b/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java
index 8702421..bb624af 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java
@@ -29,7 +29,7 @@ public class CarbonRow implements Serializable {
 
   private Object[] rawData;
 
-  public short bucketNumber;
+  private short rangeId;
 
   public CarbonRow(Object[] data) {
     this.data = data;
@@ -83,4 +83,12 @@ public class CarbonRow implements Serializable {
   public void setRawData(Object[] rawData) {
     this.rawData = rawData;
   }
+
+  public short getRangeId() {
+    return rangeId;
+  }
+
+  public void setRangeId(short rangeId) {
+    this.rangeId = rangeId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index 21ab797..a15804e 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -293,7 +293,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
       thriftColumnSchema.add(fromWrapperToExternalColumnSchema(wrapperColumnSchema));
     }
     return new org.apache.carbondata.format.BucketingInfo(thriftColumnSchema,
-        bucketingInfo.getNumberOfBuckets());
+        bucketingInfo.getNumOfRanges());
   }
 
   /* (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/core/src/main/java/org/apache/carbondata/core/metadata/schema/BucketingInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/BucketingInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/BucketingInfo.java
index 569241d..e24f0f8 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/BucketingInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/BucketingInfo.java
@@ -24,40 +24,41 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.core.metadata.schema.table.Writable;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 
 /**
  * Bucketing information
  */
-public class BucketingInfo implements Serializable, Writable {
-
+@InterfaceAudience.Internal
+public class BucketingInfo implements ColumnRangeInfo, Serializable, Writable {
   private static final long serialVersionUID = -0L;
-
   private List<ColumnSchema> listOfColumns;
-
-  private int numberOfBuckets;
+  // number of value ranges
+  private int numOfRanges;
 
   public BucketingInfo() {
 
   }
 
-  public BucketingInfo(List<ColumnSchema> listOfColumns, int numberOfBuckets) {
+  public BucketingInfo(List<ColumnSchema> listOfColumns, int numberOfRanges) {
     this.listOfColumns = listOfColumns;
-    this.numberOfBuckets = numberOfBuckets;
+    this.numOfRanges = numberOfRanges;
   }
 
   public List<ColumnSchema> getListOfColumns() {
     return listOfColumns;
   }
 
-  public int getNumberOfBuckets() {
-    return numberOfBuckets;
+  @Override
+  public int getNumOfRanges() {
+    return numOfRanges;
   }
 
   @Override
   public void write(DataOutput output) throws IOException {
-    output.writeInt(numberOfBuckets);
+    output.writeInt(numOfRanges);
     output.writeInt(listOfColumns.size());
     for (ColumnSchema aColSchema : listOfColumns) {
       aColSchema.write(output);
@@ -66,7 +67,7 @@ public class BucketingInfo implements Serializable, Writable {
 
   @Override
   public void readFields(DataInput input) throws IOException {
-    this.numberOfBuckets = input.readInt();
+    this.numOfRanges = input.readInt();
     int colSchemaSize = input.readInt();
     this.listOfColumns = new ArrayList<>(colSchemaSize);
     for (int i = 0; i < colSchemaSize; i++) {
@@ -75,5 +76,4 @@ public class BucketingInfo implements Serializable, Writable {
       this.listOfColumns.add(aSchema);
     }
   }
-
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/core/src/main/java/org/apache/carbondata/core/metadata/schema/ColumnRangeInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/ColumnRangeInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/ColumnRangeInfo.java
new file mode 100644
index 0000000..c5454b2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/ColumnRangeInfo.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.metadata.schema;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+
+/**
+ * interface for column range information. Currently we treat bucket and sort_column_range as
+ * value ranges for a column.
+ */
+@InterfaceAudience.Internal
+public interface ColumnRangeInfo {
+  int getNumOfRanges();
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/core/src/main/java/org/apache/carbondata/core/metadata/schema/SortColumnRangeInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/SortColumnRangeInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/SortColumnRangeInfo.java
new file mode 100644
index 0000000..9d2460a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/SortColumnRangeInfo.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.metadata.schema;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+
+/**
+ * column ranges specified by sort column bounds
+ */
+@InterfaceAudience.Internal
+public class SortColumnRangeInfo implements ColumnRangeInfo, Serializable {
+  private static final long serialVersionUID = 1L;
+  // indices for the sort columns in the raw row
+  private int[] sortColumnIndex;
+  // is the sort column no dictionary encoded
+  private boolean[] isSortColumnNoDict;
+  // each literal sort column bounds specified by user
+  private String[] userSpecifiedRanges;
+  // separator for the field values in each bound
+  private String separator;
+  // number of value ranges for the columns
+  private int numOfRanges;
+
+  public SortColumnRangeInfo(int[] sortColumnIndex, boolean[] isSortColumnNoDict,
+      String[] userSpecifiedRanges, String separator) {
+    this.sortColumnIndex = sortColumnIndex;
+    this.isSortColumnNoDict = isSortColumnNoDict;
+    this.userSpecifiedRanges = userSpecifiedRanges;
+    this.separator = separator;
+    this.numOfRanges = userSpecifiedRanges.length + 1;
+  }
+
+  public int[] getSortColumnIndex() {
+    return sortColumnIndex;
+  }
+
+  public boolean[] getIsSortColumnNoDict() {
+    return isSortColumnNoDict;
+  }
+
+  public String[] getUserSpecifiedRanges() {
+    return userSpecifiedRanges;
+  }
+
+  public String getSeparator() {
+    return separator;
+  }
+
+  @Override
+  public int getNumOfRanges() {
+    return numOfRanges;
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("SortColumnRangeInfo{");
+    sb.append("sortColumnIndex=").append(Arrays.toString(sortColumnIndex));
+    sb.append(", isSortColumnNoDict=").append(Arrays.toString(isSortColumnNoDict));
+    sb.append(", userSpecifiedRanges=").append(Arrays.toString(userSpecifiedRanges));
+    sb.append(", separator='").append(separator).append('\'');
+    sb.append(", numOfRanges=").append(numOfRanges);
+    sb.append('}');
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/docs/data-management-on-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/data-management-on-carbondata.md b/docs/data-management-on-carbondata.md
index 9678a32..e71e489 100644
--- a/docs/data-management-on-carbondata.md
+++ b/docs/data-management-on-carbondata.md
@@ -419,6 +419,17 @@ This tutorial is going to introduce all commands and data operations on CarbonDa
     ```
     NOTE: Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to [SimpleDateFormat](http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html).
 
+  - **SORT COLUMN BOUNDS:** Range bounds for sort columns.
+
+    ```
+    OPTIONS('SORT_COLUMN_BOUNDS'='v11,v21,v31;v12,v22,v32;v13,v23,v33')
+    ```
+    NOTE:
+    * SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.
+    * Each bound is separated by ';' and each field value in bound is separated by ','.
+    * Carbondata will use these bounds as ranges to process data concurrently.
+    * Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.
+
   - **SINGLE_PASS:** Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary.
 
   This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSortColumnBounds.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSortColumnBounds.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSortColumnBounds.scala
new file mode 100644
index 0000000..1f171b8
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithSortColumnBounds.scala
@@ -0,0 +1,348 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.integration.spark.testsuite.dataload
+
+import java.io.{File, FileOutputStream, OutputStreamWriter, Serializable}
+
+import scala.util.Random
+
+import org.apache.spark.sql.test.util.QueryTest
+import org.apache.spark.sql.{DataFrame, Row, SaveMode}
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+
+case class SortColumnBoundRow (id: Int, date: String, country: String, name: String,
+    phoneType: String, serialName: String, salary: Int) extends Serializable
+
+object TestLoadDataWithSortColumnBounds {
+  def generateOneRow(id : Int): SortColumnBoundRow = {
+    SortColumnBoundRow(id,
+      "2015/7/23",
+      s"country$id",
+      s"name$id",
+      s"phone${new Random().nextInt(10000)}",
+      s"ASD${new Random().nextInt(10000)}",
+      10000 + id)
+  }
+}
+
+class TestLoadDataWithSortColumnBounds extends QueryTest with BeforeAndAfterAll {
+  private val tableName: String = "test_table_with_sort_column_bounds"
+  private val filePath: String = s"$resourcesPath/source_for_sort_column_bounds.csv"
+  private var df: DataFrame = _
+
+  private val dateFormatStr: String = "yyyy/MM/dd"
+  private val totalLineNum = 2000
+
+  private val originDateStatus: String = CarbonProperties.getInstance().getProperty(
+    CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+    CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+
+
+  override def beforeAll(): Unit = {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, dateFormatStr)
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    prepareDataFile()
+    prepareDataFrame()
+  }
+
+  override def afterAll(): Unit = {
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, originDateStatus)
+    sql(s"DROP TABLE IF EXISTS $tableName")
+    new File(filePath).delete()
+    df = null
+  }
+
+  /**
+   * generate loading files based on source.csv but can have more lines
+   */
+  private def prepareDataFile(): Unit = {
+    val file = new File(filePath)
+
+    val sb: StringBuilder = new StringBuilder
+    def generateLine(id : Int): String = {
+      sb.clear()
+      val row = TestLoadDataWithSortColumnBounds.generateOneRow(id)
+      sb.append(row.id).append(',')
+        .append(row.date).append(',')
+        .append(row.country).append(',')
+        .append(row.name).append(',')
+        .append(row.phoneType).append(',')
+        .append(row.serialName).append(',')
+        .append(row.salary)
+        .append(System.lineSeparator())
+        .toString()
+    }
+
+    val outputStream = new FileOutputStream(file)
+    val writer = new OutputStreamWriter(outputStream)
+    for (i <- 1 to totalLineNum) {
+      writer.write(generateLine(i))
+    }
+
+    writer.flush()
+    writer.close()
+    outputStream.flush()
+    outputStream.close()
+  }
+
+  /**
+   * prepare data frame
+   */
+  private def prepareDataFrame(): Unit = {
+    import sqlContext.implicits._
+    df = sqlContext.sparkSession.sparkContext.parallelize(1 to totalLineNum)
+      .map(id => {
+        val row = TestLoadDataWithSortColumnBounds.generateOneRow(id)
+        (row.id, row.date, row.country, row.name, row.phoneType, row.serialName, row.salary)
+      })
+      .toDF("ID", "date", "country", "name", "phoneType", "serialName", "salary")
+  }
+
+  test("load data with sort column bounds: safe mode") {
+    val originStatus = CarbonProperties.getInstance().getProperty(
+      CarbonCommonConstants.ENABLE_UNSAFE_SORT, CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT)
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "false")
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata' " +
+        "tblproperties('sort_columns'='ID,name')")
+    // load with 4 bounds
+    sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+        s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+        s" 'sort_column_bounds'='400,aab1;800,aab1;1200,aab1;1600,aab1')")
+
+    checkAnswer(sql(s"select count(*) from $tableName"), Row(totalLineNum))
+    checkAnswer(sql(s"select count(*) from $tableName where ID > 1001"), Row(totalLineNum - 1001))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.ENABLE_UNSAFE_SORT, originStatus)
+  }
+
+  test("load data with sort column bounds: unsafe mode") {
+    val originStatus = CarbonProperties.getInstance().getProperty(
+      CarbonCommonConstants.ENABLE_UNSAFE_SORT, CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT)
+    CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "true")
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata' " +
+        "tblproperties('sort_columns'='ID,name')")
+    // load with 4 bounds
+    sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+        s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+        s" 'sort_column_bounds'='400,aab1;800,aab1;1200,aab1;1600,aab1')")
+
+    checkAnswer(sql(s"select count(*) from $tableName"), Row(totalLineNum))
+    checkAnswer(sql(s"select count(*) from $tableName where ID > 1001"), Row(totalLineNum - 1001))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.ENABLE_UNSAFE_SORT, originStatus)
+  }
+
+  test("load data with sort column bounds: empty column value in bounds is treated as null") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata' " +
+        "tblproperties('sort_columns'='ID,name')")
+    // bounds have empty value
+    sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+        s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+        s" 'sort_column_bounds'='200,aab1;,aab1')")
+
+    checkAnswer(sql(s"select count(*) from $tableName"), Row(totalLineNum))
+    checkAnswer(sql(s"select count(*) from $tableName where ID > 1001"), Row(totalLineNum - 1001))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+
+  test("load data with sort column bounds: sort column bounds will be ignored if it is empty.") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata' " +
+        "tblproperties('sort_columns'='ID,name')")
+    sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+        s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+        s" 'sort_column_bounds'='')")
+
+    checkAnswer(sql(s"select count(*) from $tableName"), Row(totalLineNum))
+    checkAnswer(sql(s"select count(*) from $tableName where ID > 1001"), Row(totalLineNum - 1001))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+
+  test("load data with sort column bounds: number of column value in bounds should match that of sort column") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata' " +
+        "tblproperties('sort_columns'='ID,name')")
+    val e = intercept[Exception] {
+      // number of column value does not match that of sort columns
+      sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+          s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+          s" 'sort_column_bounds'='400,aab1;800')")
+    }
+
+    assert(e.getMessage.contains(
+      "The number of field in bounds should be equal to that in sort columns." +
+      " Expected 2, actual 1." +
+      " The illegal bound is '800'"))
+
+    val e2 = intercept[Exception] {
+      // number of column value does not match that of sort columns
+      sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+          s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+          s" 'sort_column_bounds'='400,aab1;800,aab1,def')")
+    }
+
+    assert(e2.getMessage.contains(
+      "The number of field in bounds should be equal to that in sort columns." +
+      " Expected 2, actual 3." +
+      " The illegal bound is '800,aab1,def'"))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+
+  test("load data with sort column bounds: sort column bounds will be ignored if not using local_sort") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata'" +
+        "tblproperties('sort_columns'='ID,name','sort_scope'='global_sort')")
+    // since the sort_scope is 'global_sort', we will ignore the sort column bounds,
+    // so the error in sort_column bounds will not be thrown
+    sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+        s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+        s" 'sort_column_bounds'='400,aab,extra_field')")
+
+    checkAnswer(sql(s"select count(*) from $tableName"), Row(totalLineNum))
+    checkAnswer(sql(s"select count(*) from $tableName where ID > 1001"), Row(totalLineNum - 1001))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+
+  test("load data with sort column bounds: no sort columns explicitly specified" +
+       " means all dimension columns will be sort columns, so bounds should be set correctly") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata'")
+    // the sort_columns will have 5 columns if we don't specify it explicitly
+    val e = intercept[Exception] {
+      sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+          s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+          s" 'sort_column_bounds'='400,aab')")
+    }
+    assert(e.getMessage.contains(
+      "The number of field in bounds should be equal to that in sort columns." +
+      " Expected 5, actual 2." +
+      " The illegal bound is '400,aab'"))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+
+  test("load data with sort column bounds: sort column is global dictionary encoded") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata' " +
+        "tblproperties('sort_columns'='ID,name','dictionary_include'='ID')")
+    // ID is sort column and dictionary column. Since the actual order and literal order of
+    // this column are not necessarily the same, this will not cause error but will cause data skewed.
+    sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+        s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+        s" 'sort_column_bounds'='400,name400;800,name800;1200,name1200;1600,name1600')")
+
+    checkAnswer(sql(s"select count(*) from $tableName"), Row(totalLineNum))
+    checkAnswer(sql(s"select count(*) from $tableName where ID > 1001"), Row(totalLineNum - 1001))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+
+  test("load data with sort column bounds: sort column is global dictionary encoded" +
+       " but bounds are not in dictionary") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    sql(s"CREATE TABLE $tableName (ID Int, date Timestamp, country String, name String, phonetype String," +
+        "serialname String, salary Int) STORED BY 'carbondata' " +
+        "tblproperties('sort_columns'='name,ID','dictionary_include'='name')")
+    // 'name' is sort column and dictionary column, but value for 'name' in bounds does not exists
+    // in dictionary. It will not cause error but will cause data skewed.
+    sql(s"LOAD DATA INPATH '$filePath' INTO TABLE $tableName " +
+        s" OPTIONS('fileheader'='ID,date,country,name,phonetype,serialname,salary'," +
+        s" 'sort_column_bounds'='nmm400,400;nme800,800;nme1200,1200;nme1600,1600')")
+
+    checkAnswer(sql(s"select count(*) from $tableName"), Row(totalLineNum))
+    checkAnswer(sql(s"select count(*) from $tableName where ID > 1001"), Row(totalLineNum - 1001))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+
+  test("load data frame with sort column bounds") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    df.write
+      .format("carbondata")
+      .option("tableName", tableName)
+      .option("tempCSV", "false")
+      .option("sort_columns", "ID,name")
+      .option("sort_column_bounds", "600,aab1;1200,aab1")
+      .mode(SaveMode.Overwrite)
+      .save()
+
+    sql(s"select count(*) from $tableName").show()
+    sql(s"select count(*) from $tableName where ID > 1001").show()
+
+    checkAnswer(sql(s"select count(*) from $tableName"), Row(totalLineNum))
+    checkAnswer(sql(s"select count(*) from $tableName where ID > 1001"), Row(totalLineNum - 1001))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+
+  test("load data frame with sort column bounds: number of column value in bounds should match that of sort column") {
+    sql(s"DROP TABLE IF EXISTS $tableName")
+
+    val e = intercept[Exception] {
+      df.write
+        .format("carbondata")
+        .option("tableName", tableName)
+        .option("tempCSV", "false")
+        .option("sort_columns", "ID,name")
+        .option("sort_column_bounds", "600,aab1;1200,aab1,def")
+        .mode(SaveMode.Overwrite)
+        .save()
+    }
+    assert(e.getMessage.contains(
+      "The number of field in bounds should be equal to that in sort columns." +
+      " Expected 2, actual 3." +
+      " The illegal bound is '1200,aab1,def'"))
+
+    sql(s"DROP TABLE IF EXISTS $tableName")
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index ddb9b32..49a8023 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -163,7 +163,7 @@ class CarbonScanRDD(
         var i = 0
         val bucketed =
           splits.asScala.map(_.asInstanceOf[CarbonInputSplit]).groupBy(f => f.getBucketId)
-        (0 until bucketedTable.getNumberOfBuckets).map { bucketId =>
+        (0 until bucketedTable.getNumOfRanges).map { bucketId =>
           val bucketPartitions = bucketed.getOrElse(bucketId.toString, Nil)
           val multiBlockSplit =
             new CarbonMultiBlockSplit(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
index c73065d..3495885 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
@@ -50,7 +50,7 @@ object PartitionDropper {
     val bucketInfo = carbonTable.getBucketingInfo(tableName)
     val bucketNumber = bucketInfo match {
       case null => 1
-      case _ => bucketInfo.getNumberOfBuckets
+      case _ => bucketInfo.getNumOfRanges
     }
     val partitionIndex = oldPartitionIds.indexOf(Integer.valueOf(partitionId))
     val targetPartitionId = partitionInfo.getPartitionType match {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
index 9106cca..0d437f6 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
@@ -46,7 +46,7 @@ object PartitionSplitter {
      var finalSplitStatus = false
      val bucketNumber = bucketInfo match {
        case null => 1
-       case _ => bucketInfo.getNumberOfBuckets
+       case _ => bucketInfo.getNumOfRanges
      }
      val partitionInfo = carbonTable.getPartitionInfo(tableName)
      val partitioner = PartitionFactory.getPartitioner(partitionInfo)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index b1e4083..54ac7dc 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -881,7 +881,8 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       "SERIALIZATION_NULL_FORMAT", "BAD_RECORDS_LOGGER_ENABLE", "BAD_RECORDS_ACTION",
       "ALL_DICTIONARY_PATH", "MAXCOLUMNS", "COMMENTCHAR", "DATEFORMAT", "BAD_RECORD_PATH",
       "BATCH_SORT_SIZE_INMB", "GLOBAL_SORT_PARTITIONS", "SINGLE_PASS",
-      "IS_EMPTY_DATA_BAD_RECORD", "HEADER", "TIMESTAMPFORMAT", "SKIP_EMPTY_LINE"
+      "IS_EMPTY_DATA_BAD_RECORD", "HEADER", "TIMESTAMPFORMAT", "SKIP_EMPTY_LINE",
+      "SORT_COLUMN_BOUNDS"
     )
     var isSupported = true
     val invalidOptions = StringBuilder.newBuilder

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
index 48679b1..f82d9c2 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -434,7 +434,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
       val cols = info.getListOfColumns.asScala
       val sortColumn = carbonTable.
         getDimensionByTableName(carbonTable.getTableName).get(0).getColName
-      val numBuckets = info.getNumberOfBuckets
+      val numBuckets = info.getNumOfRanges
       val bucketColumns = cols.flatMap { n =>
         val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
         attrRef match {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
index 895fb79..c21c57f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
@@ -29,6 +29,7 @@ import org.apache.carbondata.core.keygenerator.factory.KeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.schema.BucketingInfo;
+import org.apache.carbondata.core.metadata.schema.SortColumnRangeInfo;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.processing.loading.converter.DictionaryCardinalityFinder;
 
@@ -107,6 +108,8 @@ public class CarbonDataLoadConfiguration {
    */
   private short writingCoresCount;
 
+  private SortColumnRangeInfo sortColumnRangeInfo;
+
   /**
    * Flder path to where data should be written for this load.
    */
@@ -366,4 +369,12 @@ public class CarbonDataLoadConfiguration {
   public void setDataWritePath(String dataWritePath) {
     this.dataWritePath = dataWritePath;
   }
+
+  public SortColumnRangeInfo getSortColumnRangeInfo() {
+    return sortColumnRangeInfo;
+  }
+
+  public void setSortColumnRangeInfo(SortColumnRangeInfo sortColumnRangeInfo) {
+    this.sortColumnRangeInfo = sortColumnRangeInfo;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
index fc2796a..011eeae 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
@@ -22,23 +22,26 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.datastore.TableSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.schema.SortColumnRangeInfo;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
 import org.apache.carbondata.processing.loading.steps.CarbonRowDataWriterProcessorStepImpl;
 import org.apache.carbondata.processing.loading.steps.DataConverterProcessorStepImpl;
-import org.apache.carbondata.processing.loading.steps.DataConverterProcessorWithBucketingStepImpl;
 import org.apache.carbondata.processing.loading.steps.DataWriterBatchProcessorStepImpl;
 import org.apache.carbondata.processing.loading.steps.DataWriterProcessorStepImpl;
 import org.apache.carbondata.processing.loading.steps.InputProcessorStepForPartitionImpl;
@@ -52,6 +55,8 @@ import org.apache.commons.lang3.StringUtils;
  * It builds the pipe line of steps for loading data to carbon.
  */
 public final class DataLoadProcessBuilder {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DataLoadProcessBuilder.class.getName());
 
   public AbstractDataLoadProcessorStep build(CarbonLoadModel loadModel, String[] storeLocation,
       CarbonIterator[] inputIterators) throws Exception {
@@ -150,7 +155,7 @@ public final class DataLoadProcessBuilder {
     // 2. Converts the data like dictionary or non dictionary or complex objects depends on
     // data types and configurations.
     AbstractDataLoadProcessorStep converterProcessorStep =
-        new DataConverterProcessorWithBucketingStepImpl(configuration, inputProcessorStep);
+        new DataConverterProcessorStepImpl(configuration, inputProcessorStep);
     // 3. Sorts the data by SortColumn or not
     AbstractDataLoadProcessorStep sortProcessorStep =
         new SortProcessorStepImpl(configuration, converterProcessorStep);
@@ -252,6 +257,7 @@ public final class DataLoadProcessBuilder {
     configuration.setNumberOfSortColumns(carbonTable.getNumberOfSortColumns());
     configuration.setNumberOfNoDictSortColumns(carbonTable.getNumberOfNoDictSortColumns());
     configuration.setDataWritePath(loadModel.getDataWritePath());
+    setSortColumnInfo(carbonTable, loadModel, configuration);
     // For partition loading always use single core as it already runs in multiple
     // threads per partition
     if (carbonTable.isHivePartitionTable()) {
@@ -262,4 +268,71 @@ public final class DataLoadProcessBuilder {
     return configuration;
   }
 
+  /**
+   * set sort column info in configuration
+   * @param carbonTable carbon table
+   * @param loadModel load model
+   * @param configuration configuration
+   */
+  private static void setSortColumnInfo(CarbonTable carbonTable, CarbonLoadModel loadModel,
+      CarbonDataLoadConfiguration configuration) {
+    List<String> sortCols = carbonTable.getSortColumns(carbonTable.getTableName());
+    SortScopeOptions.SortScope sortScope = SortScopeOptions.getSortScope(loadModel.getSortScope());
+    if (!SortScopeOptions.SortScope.LOCAL_SORT.equals(sortScope)
+        || sortCols.size() == 0
+        || StringUtils.isBlank(loadModel.getSortColumnsBoundsStr())) {
+      if (!StringUtils.isBlank(loadModel.getSortColumnsBoundsStr())) {
+        LOGGER.warn("sort column bounds will be ignored");
+      }
+
+      configuration.setSortColumnRangeInfo(null);
+      return;
+    }
+    // column index for sort columns
+    int[] sortColIndex = new int[sortCols.size()];
+    boolean[] isSortColNoDict = new boolean[sortCols.size()];
+
+    DataField[] outFields = configuration.getDataFields();
+    int j = 0;
+    boolean columnExist;
+    for (String sortCol : sortCols) {
+      columnExist = false;
+
+      for (int i = 0; !columnExist && i < outFields.length; i++) {
+        if (outFields[i].getColumn().getColName().equalsIgnoreCase(sortCol)) {
+          columnExist = true;
+
+          sortColIndex[j] = i;
+          isSortColNoDict[j] = !outFields[i].hasDictionaryEncoding();
+          j++;
+        }
+      }
+
+      if (!columnExist) {
+        throw new CarbonDataLoadingException("Field " + sortCol + " does not exist.");
+      }
+    }
+
+    String[] sortColumnBounds = StringUtils.splitPreserveAllTokens(
+        loadModel.getSortColumnsBoundsStr(),
+        CarbonLoadOptionConstants.SORT_COLUMN_BOUNDS_ROW_DELIMITER, -1);
+    for (String bound : sortColumnBounds) {
+      String[] fieldInBounds = StringUtils.splitPreserveAllTokens(bound,
+          CarbonLoadOptionConstants.SORT_COLUMN_BOUNDS_FIELD_DELIMITER, -1);
+      if (fieldInBounds.length != sortCols.size()) {
+        String msg = new StringBuilder(
+            "The number of field in bounds should be equal to that in sort columns.")
+            .append(" Expected ").append(sortCols.size())
+            .append(", actual ").append(String.valueOf(fieldInBounds.length)).append(".")
+            .append(" The illegal bound is '").append(bound).append("'.").toString();
+        throw new CarbonDataLoadingException(msg);
+      }
+    }
+
+    SortColumnRangeInfo sortColumnRangeInfo = new SortColumnRangeInfo(sortColIndex,
+        isSortColNoDict,
+        sortColumnBounds,
+        CarbonLoadOptionConstants.SORT_COLUMN_BOUNDS_FIELD_DELIMITER);
+    configuration.setSortColumnRangeInfo(sortColumnRangeInfo);
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java
index fd3a650..016ff3f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.java
@@ -31,6 +31,6 @@ public interface RowConverter extends DictionaryCardinalityFinder {
   CarbonRow convert(CarbonRow row) throws CarbonDataLoadingException;
 
   RowConverter createCopyForNewThread();
-
+  FieldConverter[] getFieldConverters();
   void finish();
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
index c5313cb..208d42f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
@@ -250,4 +250,9 @@ public class RowConverterImpl implements RowConverter {
     }
     return cardinality;
   }
+
+  @Override
+  public FieldConverter[] getFieldConverters() {
+    return fieldConverters;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
index 890492e..e516bc5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -189,6 +189,10 @@ public class CarbonLoadModel implements Serializable {
   private String globalSortPartitions;
 
   private boolean isAggLoadRequest;
+  /**
+   * sort columns bounds
+   */
+  private String sortColumnsBoundsStr;
 
   /**
    * It directly writes data directly to nosort processor bypassing all other processors.
@@ -367,6 +371,14 @@ public class CarbonLoadModel implements Serializable {
     this.dictionaryServiceProvider = dictionaryServiceProvider;
   }
 
+  public String getSortColumnsBoundsStr() {
+    return sortColumnsBoundsStr;
+  }
+
+  public void setSortColumnsBoundsStr(String sortColumnsBoundsStr) {
+    this.sortColumnsBoundsStr = sortColumnsBoundsStr;
+  }
+
   /**
    * get copy with partition
    *
@@ -468,6 +480,7 @@ public class CarbonLoadModel implements Serializable {
     copy.isAggLoadRequest = isAggLoadRequest;
     copy.badRecordsLocation = badRecordsLocation;
     copy.isPartitionLoad = isPartitionLoad;
+    copy.sortColumnsBoundsStr = sortColumnsBoundsStr;
     return copy;
   }
 
@@ -524,6 +537,7 @@ public class CarbonLoadModel implements Serializable {
     copyObj.batchSortSizeInMb = batchSortSizeInMb;
     copyObj.badRecordsLocation = badRecordsLocation;
     copyObj.isAggLoadRequest = isAggLoadRequest;
+    copyObj.sortColumnsBoundsStr = sortColumnsBoundsStr;
     return copyObj;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
index 99684ad..17e8dbe 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -223,6 +223,7 @@ public class CarbonLoadModelBuilder {
 
     carbonLoadModel.setMaxColumns(String.valueOf(validatedMaxColumns));
     carbonLoadModel.readAndSetLoadMetadataDetails();
+    carbonLoadModel.setSortColumnsBoundsStr(optionsFinal.get("sort_column_bounds"));
   }
 
   private int validateMaxColumns(String[] csvHeaders, String maxColumns)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
index bd942ca..5af4859 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
@@ -191,6 +191,7 @@ public class LoadOption {
     }
 
     optionsFinal.put("single_pass", String.valueOf(singlePass));
+    optionsFinal.put("sort_column_bounds", Maps.getOrDefault(options, "sort_column_bounds", ""));
     return optionsFinal;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
index f24d24f..e10faf6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.processing.loading.partition.impl;
 
 import java.util.List;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -27,7 +29,8 @@ import org.apache.carbondata.processing.loading.partition.Partitioner;
 /**
  * Hash partitioner implementation
  */
-public class HashPartitionerImpl implements Partitioner<Object[]> {
+@InterfaceAudience.Internal
+public class HashPartitionerImpl implements Partitioner<CarbonRow> {
 
   private int numberOfBuckets;
 
@@ -50,10 +53,11 @@ public class HashPartitionerImpl implements Partitioner<Object[]> {
     }
   }
 
-  @Override public int getPartition(Object[] objects) {
+  @Override
+  public int getPartition(CarbonRow key) {
     int hashCode = 0;
     for (Hash hash : hashes) {
-      hashCode += hash.getHash(objects);
+      hashCode += hash.getHash(key.getData());
     }
     return (hashCode & Integer.MAX_VALUE) % numberOfBuckets;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RangePartitionerImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RangePartitionerImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RangePartitionerImpl.java
new file mode 100644
index 0000000..d59ad02
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RangePartitionerImpl.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.partition.impl;
+
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.processing.loading.partition.Partitioner;
+
+@InterfaceAudience.Internal
+public class RangePartitionerImpl implements Partitioner<CarbonRow> {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(RangePartitionerImpl.class.getName());
+  private CarbonRow[] rangeBounds;
+  private Comparator<CarbonRow> comparator;
+
+  public RangePartitionerImpl(CarbonRow[] rangeBounds, Comparator<CarbonRow> comparator) {
+    this.rangeBounds = rangeBounds;
+    LOGGER.info("Use range partitioner to distribute data to "
+        + (rangeBounds.length + 1) + " ranges.");
+    this.comparator = comparator;
+  }
+
+  /**
+   * learned from spark org.apache.spark.RangePartitioner
+   *
+   * @param key key
+   * @return partitionId
+   */
+  @Override
+  public int getPartition(CarbonRow key) {
+    int partition = 0;
+    if (rangeBounds.length <= 128) {
+      // If we have less than 128 partitions naive search
+      while (partition < rangeBounds.length
+          && comparator.compare(key, rangeBounds[partition]) > 0) {
+        partition += 1;
+      }
+    } else {
+      // binary search. binarySearch either returns the match location or -[insertion point]-1
+      partition = Arrays.binarySearch(rangeBounds, 0, rangeBounds.length, key, comparator);
+      if (partition < 0) {
+        partition = -partition - 1;
+      }
+      if (partition > rangeBounds.length) {
+        partition = rangeBounds.length;
+      }
+    }
+
+    return partition;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java
new file mode 100644
index 0000000..64b64f5
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/RawRowComparator.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.partition.impl;
+
+import java.util.Comparator;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
+
+/**
+ * comparator for the converted row. The row has not been rearranged as 3-parted yet.
+ */
+@InterfaceAudience.Internal
+public class RawRowComparator implements Comparator<CarbonRow> {
+  private int[] sortColumnIndices;
+  private boolean[] isSortColumnNoDict;
+
+  public RawRowComparator(int[] sortColumnIndices, boolean[] isSortColumnNoDict) {
+    this.sortColumnIndices = sortColumnIndices;
+    this.isSortColumnNoDict = isSortColumnNoDict;
+  }
+
+  @Override
+  public int compare(CarbonRow o1, CarbonRow o2) {
+    int diff = 0;
+    int i = 0;
+    for (int colIdx : sortColumnIndices) {
+      if (isSortColumnNoDict[i]) {
+        byte[] colA = (byte[]) o1.getObject(colIdx);
+        byte[] colB = (byte[]) o2.getObject(colIdx);
+        diff = UnsafeComparer.INSTANCE.compareTo(colA, colB);
+        if (diff != 0) {
+          return diff;
+        }
+      } else {
+        int colA = (int) o1.getObject(colIdx);
+        int colB = (int) o2.getObject(colIdx);
+        diff = colA - colB;
+        if (diff != 0) {
+          return diff;
+        }
+      }
+      i++;
+    }
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java
index a8f0282..b74b393 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.java
@@ -25,10 +25,10 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.sort.impl.ParallelReadMergeSorterImpl;
-import org.apache.carbondata.processing.loading.sort.impl.ParallelReadMergeSorterWithBucketingImpl;
+import org.apache.carbondata.processing.loading.sort.impl.ParallelReadMergeSorterWithColumnRangeImpl;
 import org.apache.carbondata.processing.loading.sort.impl.UnsafeBatchParallelReadMergeSorterImpl;
 import org.apache.carbondata.processing.loading.sort.impl.UnsafeParallelReadMergeSorterImpl;
-import org.apache.carbondata.processing.loading.sort.impl.UnsafeParallelReadMergeSorterWithBucketingImpl;
+import org.apache.carbondata.processing.loading.sort.impl.UnsafeParallelReadMergeSorterWithColumnRangeImpl;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 public class SorterFactory {
@@ -44,15 +44,21 @@ public class SorterFactory {
     Sorter sorter;
     if (offheapsort) {
       if (configuration.getBucketingInfo() != null) {
-        sorter = new UnsafeParallelReadMergeSorterWithBucketingImpl(configuration.getDataFields(),
+        sorter = new UnsafeParallelReadMergeSorterWithColumnRangeImpl(counter,
             configuration.getBucketingInfo());
+      } else if (configuration.getSortColumnRangeInfo() != null) {
+        sorter = new UnsafeParallelReadMergeSorterWithColumnRangeImpl(counter,
+            configuration.getSortColumnRangeInfo());
       } else {
         sorter = new UnsafeParallelReadMergeSorterImpl(counter);
       }
     } else {
       if (configuration.getBucketingInfo() != null) {
-        sorter =
-            new ParallelReadMergeSorterWithBucketingImpl(counter, configuration.getBucketingInfo());
+        sorter = new ParallelReadMergeSorterWithColumnRangeImpl(counter,
+            configuration.getBucketingInfo());
+      } else if (configuration.getSortColumnRangeInfo() != null) {
+        sorter = new ParallelReadMergeSorterWithColumnRangeImpl(counter,
+            configuration.getSortColumnRangeInfo());
       } else {
         sorter = new ParallelReadMergeSorterImpl(counter);
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
deleted file mode 100644
index b7452a7..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
+++ /dev/null
@@ -1,272 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.loading.sort.impl;
-
-import java.io.File;
-import java.util.Iterator;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.metadata.schema.BucketingInfo;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
-import org.apache.carbondata.processing.loading.sort.AbstractMergeSorter;
-import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sort.sortdata.SingleThreadFinalSortFilesMerger;
-import org.apache.carbondata.processing.sort.sortdata.SortDataRows;
-import org.apache.carbondata.processing.sort.sortdata.SortIntermediateFileMerger;
-import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * It parallely reads data from array of iterates and do merge sort.
- * First it sorts the data and write to temp files. These temp files will be merge sorted to get
- * final merge sort result.
- * This step is specifically for bucketing, it sorts each bucket data separately and write to
- * temp files.
- */
-public class ParallelReadMergeSorterWithBucketingImpl extends AbstractMergeSorter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ParallelReadMergeSorterWithBucketingImpl.class.getName());
-
-  private SortParameters sortParameters;
-
-  private SortIntermediateFileMerger[] intermediateFileMergers;
-
-  private BucketingInfo bucketingInfo;
-
-  private int sortBufferSize;
-
-  private AtomicLong rowCounter;
-
-  public ParallelReadMergeSorterWithBucketingImpl(AtomicLong rowCounter,
-      BucketingInfo bucketingInfo) {
-    this.rowCounter = rowCounter;
-    this.bucketingInfo = bucketingInfo;
-  }
-
-  @Override public void initialize(SortParameters sortParameters) {
-    this.sortParameters = sortParameters;
-    int buffer = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.SORT_SIZE, CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL));
-    sortBufferSize = buffer / bucketingInfo.getNumberOfBuckets();
-    if (sortBufferSize < 100) {
-      sortBufferSize = 100;
-    }
-  }
-
-  @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
-      throws CarbonDataLoadingException {
-    SortDataRows[] sortDataRows = new SortDataRows[bucketingInfo.getNumberOfBuckets()];
-    intermediateFileMergers =
-        new SortIntermediateFileMerger[sortDataRows.length];
-    try {
-      for (int i = 0; i < bucketingInfo.getNumberOfBuckets(); i++) {
-        SortParameters parameters = sortParameters.getCopy();
-        parameters.setPartitionID(i + "");
-        setTempLocation(parameters);
-        parameters.setBufferSize(sortBufferSize);
-        intermediateFileMergers[i] = new SortIntermediateFileMerger(parameters);
-        sortDataRows[i] = new SortDataRows(parameters, intermediateFileMergers[i]);
-        sortDataRows[i].initialize();
-      }
-    } catch (CarbonSortKeyAndGroupByException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
-    this.threadStatusObserver = new ThreadStatusObserver(executorService);
-    final int batchSize = CarbonProperties.getInstance().getBatchSize();
-    try {
-      for (int i = 0; i < iterators.length; i++) {
-        executorService.execute(new SortIteratorThread(iterators[i], sortDataRows, rowCounter,
-            this.threadStatusObserver));
-      }
-      executorService.shutdown();
-      executorService.awaitTermination(2, TimeUnit.DAYS);
-      processRowToNextStep(sortDataRows, sortParameters);
-    } catch (Exception e) {
-      checkError();
-      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
-    }
-    checkError();
-    try {
-      for (int i = 0; i < intermediateFileMergers.length; i++) {
-        intermediateFileMergers[i].finish();
-      }
-    } catch (CarbonDataWriterException e) {
-      throw new CarbonDataLoadingException(e);
-    } catch (CarbonSortKeyAndGroupByException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-
-    Iterator<CarbonRowBatch>[] batchIterator = new Iterator[bucketingInfo.getNumberOfBuckets()];
-    for (int i = 0; i < bucketingInfo.getNumberOfBuckets(); i++) {
-      batchIterator[i] = new MergedDataIterator(String.valueOf(i), batchSize);
-    }
-
-    return batchIterator;
-  }
-
-  private SingleThreadFinalSortFilesMerger getFinalMerger(String bucketId) {
-    String[] storeLocation = CarbonDataProcessorUtil.getLocalDataFolderLocation(
-        sortParameters.getDatabaseName(), sortParameters.getTableName(),
-        String.valueOf(sortParameters.getTaskNo()), sortParameters.getSegmentId(),
-        false, false);
-    // Set the data file location
-    String[] dataFolderLocation = CarbonDataProcessorUtil.arrayAppend(storeLocation, File.separator,
-        CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
-    return new SingleThreadFinalSortFilesMerger(dataFolderLocation, sortParameters.getTableName(),
-            sortParameters);
-  }
-
-  @Override public void close() {
-    for (int i = 0; i < intermediateFileMergers.length; i++) {
-      intermediateFileMergers[i].close();
-    }
-  }
-
-  /**
-   * Below method will be used to process data to next step
-   */
-  private boolean processRowToNextStep(SortDataRows[] sortDataRows, SortParameters parameters)
-      throws CarbonDataLoadingException {
-    if (null == sortDataRows || sortDataRows.length == 0) {
-      LOGGER.info("Record Processed For table: " + parameters.getTableName());
-      LOGGER.info("Number of Records was Zero");
-      String logMessage = "Summary: Carbon Sort Key Step: Read: " + 0 + ": Write: " + 0;
-      LOGGER.info(logMessage);
-      return false;
-    }
-
-    try {
-      for (int i = 0; i < sortDataRows.length; i++) {
-        // start sorting
-        sortDataRows[i].startSorting();
-      }
-      // check any more rows are present
-      LOGGER.info("Record Processed For table: " + parameters.getTableName());
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordSortRowsStepTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordDictionaryValuesTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
-      return false;
-    } catch (CarbonSortKeyAndGroupByException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-  }
-
-  private void setTempLocation(SortParameters parameters) {
-    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil.getLocalDataFolderLocation(
-        parameters.getDatabaseName(), parameters.getTableName(), parameters.getTaskNo(),
-        parameters.getSegmentId(), false, false);
-    String[] tmpLocs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath, File.separator,
-        CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
-    parameters.setTempFileLocation(tmpLocs);
-  }
-
-  /**
-   * This thread iterates the iterator and adds the rows to @{@link SortDataRows}
-   */
-  private static class SortIteratorThread implements Runnable {
-
-    private Iterator<CarbonRowBatch> iterator;
-
-    private SortDataRows[] sortDataRows;
-
-    private AtomicLong rowCounter;
-
-    private ThreadStatusObserver threadStatusObserver;
-
-    public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortDataRows[] sortDataRows,
-        AtomicLong rowCounter, ThreadStatusObserver observer) {
-      this.iterator = iterator;
-      this.sortDataRows = sortDataRows;
-      this.rowCounter = rowCounter;
-      this.threadStatusObserver = observer;
-    }
-
-    @Override
-    public void run() {
-      try {
-        while (iterator.hasNext()) {
-          CarbonRowBatch batch = iterator.next();
-          int i = 0;
-          while (batch.hasNext()) {
-            CarbonRow row = batch.next();
-            if (row != null) {
-              SortDataRows sortDataRow = sortDataRows[row.bucketNumber];
-              synchronized (sortDataRow) {
-                sortDataRow.addRow(row.getData());
-                rowCounter.getAndAdd(1);
-              }
-            }
-          }
-        }
-      } catch (Exception e) {
-        LOGGER.error(e);
-        this.threadStatusObserver.notifyFailed(e);
-      }
-    }
-
-  }
-
-  private class MergedDataIterator extends CarbonIterator<CarbonRowBatch> {
-
-    private String partitionId;
-
-    private int batchSize;
-
-    private boolean firstRow = true;
-
-    public MergedDataIterator(String partitionId, int batchSize) {
-      this.partitionId = partitionId;
-      this.batchSize = batchSize;
-    }
-
-    private SingleThreadFinalSortFilesMerger finalMerger;
-
-    @Override public boolean hasNext() {
-      if (firstRow) {
-        firstRow = false;
-        finalMerger = getFinalMerger(partitionId);
-        finalMerger.startFinalMerge();
-      }
-      return finalMerger.hasNext();
-    }
-
-    @Override public CarbonRowBatch next() {
-      int counter = 0;
-      CarbonRowBatch rowBatch = new CarbonRowBatch(batchSize);
-      while (finalMerger.hasNext() && counter < batchSize) {
-        rowBatch.addRow(new CarbonRow(finalMerger.next()));
-        counter++;
-      }
-      return rowBatch;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fa1a100e/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithColumnRangeImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithColumnRangeImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithColumnRangeImpl.java
new file mode 100644
index 0000000..808952b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithColumnRangeImpl.java
@@ -0,0 +1,289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.sort.impl;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.metadata.schema.ColumnRangeInfo;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
+import org.apache.carbondata.processing.loading.sort.AbstractMergeSorter;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+import org.apache.carbondata.processing.sort.sortdata.SingleThreadFinalSortFilesMerger;
+import org.apache.carbondata.processing.sort.sortdata.SortDataRows;
+import org.apache.carbondata.processing.sort.sortdata.SortIntermediateFileMerger;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+/**
+ * It parallely reads data from array of iterates and do merge sort.
+ * First it sorts the data and write to temp files. These temp files will be merge sorted to get
+ * final merge sort result.
+ * This step is specifically for the data loading with specifying column value range, such as
+ * bucketing,sort_column_bounds, it sorts each range of data separately and write to temp files.
+ */
+public class ParallelReadMergeSorterWithColumnRangeImpl extends AbstractMergeSorter {
+  private static final LogService LOGGER = LogServiceFactory.getLogService(
+      ParallelReadMergeSorterWithColumnRangeImpl.class.getName());
+
+  private SortParameters originSortParameters;
+
+  private SortIntermediateFileMerger[] intermediateFileMergers;
+
+  private ColumnRangeInfo columnRangeInfo;
+
+  private int sortBufferSize;
+
+  private AtomicLong rowCounter;
+  /**
+   * counters to collect information about rows processed by each range
+   */
+  private List<AtomicLong> insideRowCounterList;
+
+  public ParallelReadMergeSorterWithColumnRangeImpl(AtomicLong rowCounter,
+      ColumnRangeInfo columnRangeInfo) {
+    this.rowCounter = rowCounter;
+    this.columnRangeInfo = columnRangeInfo;
+  }
+
+  @Override
+  public void initialize(SortParameters sortParameters) {
+    this.originSortParameters = sortParameters;
+    int buffer = Integer.parseInt(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.SORT_SIZE, CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL));
+    sortBufferSize = buffer / columnRangeInfo.getNumOfRanges();
+    if (sortBufferSize < 100) {
+      sortBufferSize = 100;
+    }
+    this.insideRowCounterList = new ArrayList<>(columnRangeInfo.getNumOfRanges());
+    for (int i = 0; i < columnRangeInfo.getNumOfRanges(); i++) {
+      insideRowCounterList.add(new AtomicLong(0));
+    }
+  }
+
+  @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
+      throws CarbonDataLoadingException {
+    SortDataRows[] sortDataRows = new SortDataRows[columnRangeInfo.getNumOfRanges()];
+    intermediateFileMergers = new SortIntermediateFileMerger[columnRangeInfo.getNumOfRanges()];
+    SortParameters[] sortParameterArray = new SortParameters[columnRangeInfo.getNumOfRanges()];
+    try {
+      for (int i = 0; i < columnRangeInfo.getNumOfRanges(); i++) {
+        SortParameters parameters = originSortParameters.getCopy();
+        parameters.setPartitionID(i + "");
+        parameters.setRangeId(i);
+        sortParameterArray[i] = parameters;
+        setTempLocation(parameters);
+        parameters.setBufferSize(sortBufferSize);
+        intermediateFileMergers[i] = new SortIntermediateFileMerger(parameters);
+        sortDataRows[i] = new SortDataRows(parameters, intermediateFileMergers[i]);
+        sortDataRows[i].initialize();
+      }
+    } catch (CarbonSortKeyAndGroupByException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
+    this.threadStatusObserver = new ThreadStatusObserver(executorService);
+    final int batchSize = CarbonProperties.getInstance().getBatchSize();
+    try {
+      // dispatch rows to sortDataRows by range id
+      for (int i = 0; i < iterators.length; i++) {
+        executorService.execute(new SortIteratorThread(iterators[i], sortDataRows, rowCounter,
+            this.insideRowCounterList, this.threadStatusObserver));
+      }
+      executorService.shutdown();
+      executorService.awaitTermination(2, TimeUnit.DAYS);
+      processRowToNextStep(sortDataRows, originSortParameters);
+    } catch (Exception e) {
+      checkError();
+      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
+    }
+    checkError();
+    try {
+      for (int i = 0; i < intermediateFileMergers.length; i++) {
+        intermediateFileMergers[i].finish();
+      }
+    } catch (CarbonDataWriterException e) {
+      throw new CarbonDataLoadingException(e);
+    } catch (CarbonSortKeyAndGroupByException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+
+    Iterator<CarbonRowBatch>[] batchIterator = new Iterator[columnRangeInfo.getNumOfRanges()];
+    for (int i = 0; i < columnRangeInfo.getNumOfRanges(); i++) {
+      batchIterator[i] = new MergedDataIterator(sortParameterArray[i], batchSize);
+    }
+
+    return batchIterator;
+  }
+
+  private SingleThreadFinalSortFilesMerger getFinalMerger(SortParameters sortParameters) {
+    String[] storeLocation = CarbonDataProcessorUtil
+        .getLocalDataFolderLocation(sortParameters.getDatabaseName(), sortParameters.getTableName(),
+            String.valueOf(sortParameters.getTaskNo()),
+            sortParameters.getSegmentId() + "", false, false);
+    // Set the data file location
+    String[] dataFolderLocation = CarbonDataProcessorUtil.arrayAppend(storeLocation, File.separator,
+        CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
+    return new SingleThreadFinalSortFilesMerger(dataFolderLocation, sortParameters.getTableName(),
+        sortParameters);
+  }
+
+  @Override public void close() {
+    for (int i = 0; i < intermediateFileMergers.length; i++) {
+      intermediateFileMergers[i].close();
+    }
+  }
+
+  /**
+   * Below method will be used to process data to next step
+   */
+  private boolean processRowToNextStep(SortDataRows[] sortDataRows, SortParameters parameters)
+      throws CarbonDataLoadingException {
+    if (null == sortDataRows || sortDataRows.length == 0) {
+      LOGGER.info("Record Processed For table: " + parameters.getTableName());
+      LOGGER.info("Number of Records was Zero");
+      String logMessage = "Summary: Carbon Sort Key Step: Read: " + 0 + ": Write: " + 0;
+      LOGGER.info(logMessage);
+      return false;
+    }
+
+    try {
+      for (int i = 0; i < sortDataRows.length; i++) {
+        // start sorting
+        sortDataRows[i].startSorting();
+      }
+      // check any more rows are present
+      LOGGER.info("Record Processed For table: " + parameters.getTableName());
+      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+          .recordSortRowsStepTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
+      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+          .recordDictionaryValuesTotalTime(parameters.getPartitionID(), System.currentTimeMillis());
+      return false;
+    } catch (CarbonSortKeyAndGroupByException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+  }
+
+  private void setTempLocation(SortParameters parameters) {
+    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
+        .getLocalDataFolderLocation(parameters.getDatabaseName(),
+            parameters.getTableName(), parameters.getTaskNo(),
+            parameters.getSegmentId(), false, false);
+    String[] tmpLocs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath, File.separator,
+        CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
+    parameters.setTempFileLocation(tmpLocs);
+  }
+
+  /**
+   * This thread iterates the iterator and adds the rows to @{@link SortDataRows}
+   */
+  private static class SortIteratorThread implements Runnable {
+
+    private Iterator<CarbonRowBatch> iterator;
+
+    private SortDataRows[] sortDataRows;
+
+    private AtomicLong rowCounter;
+    private List<AtomicLong> insideCounterList;
+    private ThreadStatusObserver threadStatusObserver;
+
+    public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortDataRows[] sortDataRows,
+        AtomicLong rowCounter, List<AtomicLong> insideCounterList,
+        ThreadStatusObserver observer) {
+      this.iterator = iterator;
+      this.sortDataRows = sortDataRows;
+      this.rowCounter = rowCounter;
+      this.insideCounterList = insideCounterList;
+      this.threadStatusObserver = observer;
+    }
+
+    @Override
+    public void run() {
+      try {
+        while (iterator.hasNext()) {
+          CarbonRowBatch batch = iterator.next();
+          while (batch.hasNext()) {
+            CarbonRow row = batch.next();
+            if (row != null) {
+              SortDataRows sortDataRow = sortDataRows[row.getRangeId()];
+              synchronized (sortDataRow) {
+                sortDataRow.addRow(row.getData());
+                insideCounterList.get(row.getRangeId()).getAndIncrement();
+                rowCounter.getAndAdd(1);
+              }
+            }
+          }
+        }
+        LOGGER.info("Rows processed by each range: " + insideCounterList);
+      } catch (Exception e) {
+        LOGGER.error(e);
+        this.threadStatusObserver.notifyFailed(e);
+      }
+    }
+
+  }
+
+  private class MergedDataIterator extends CarbonIterator<CarbonRowBatch> {
+
+    private SortParameters sortParameters;
+
+    private int batchSize;
+
+    private boolean firstRow = true;
+
+    public MergedDataIterator(SortParameters sortParameters, int batchSize) {
+      this.sortParameters = sortParameters;
+      this.batchSize = batchSize;
+    }
+
+    private SingleThreadFinalSortFilesMerger finalMerger;
+
+    @Override public boolean hasNext() {
+      if (firstRow) {
+        firstRow = false;
+        finalMerger = getFinalMerger(sortParameters);
+        finalMerger.startFinalMerge();
+      }
+      return finalMerger.hasNext();
+    }
+
+    @Override public CarbonRowBatch next() {
+      int counter = 0;
+      CarbonRowBatch rowBatch = new CarbonRowBatch(batchSize);
+      while (finalMerger.hasNext() && counter < batchSize) {
+        rowBatch.addRow(new CarbonRow(finalMerger.next()));
+        counter++;
+      }
+      return rowBatch;
+    }
+  }
+}


[45/49] carbondata git commit: [CARBONDATA-2186] Add InterfaceAudience.Internal to annotate internal interface

Posted by ja...@apache.org.
[CARBONDATA-2186] Add InterfaceAudience.Internal to annotate internal interface

This closes #1986


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

Branch: refs/heads/carbonstore-rebase4
Commit: e059167147a9b4db8d67cd7ef05549abf277010a
Parents: 33a6d2b
Author: Jacky Li <ja...@qq.com>
Authored: Tue Feb 20 11:16:53 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:20:57 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/carbondata/common/Maps.java  |  2 +-
 .../org/apache/carbondata/common/Strings.java    |  2 +-
 .../common/annotations/InterfaceAudience.java    | 19 ++++++++++++++-----
 .../common/annotations/InterfaceStability.java   |  2 +-
 .../loading/model/CarbonLoadModelBuilder.java    |  2 +-
 .../processing/loading/model/LoadOption.java     |  2 +-
 .../carbondata/sdk/file/CSVCarbonWriter.java     |  4 +---
 7 files changed, 20 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/e0591671/common/src/main/java/org/apache/carbondata/common/Maps.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/Maps.java b/common/src/main/java/org/apache/carbondata/common/Maps.java
index 14fc329..4e76192 100644
--- a/common/src/main/java/org/apache/carbondata/common/Maps.java
+++ b/common/src/main/java/org/apache/carbondata/common/Maps.java
@@ -21,7 +21,7 @@ import java.util.Map;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 
-@InterfaceAudience.Developer
+@InterfaceAudience.Internal
 public class Maps {
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e0591671/common/src/main/java/org/apache/carbondata/common/Strings.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/Strings.java b/common/src/main/java/org/apache/carbondata/common/Strings.java
index 08fdc3c..23c7f9f 100644
--- a/common/src/main/java/org/apache/carbondata/common/Strings.java
+++ b/common/src/main/java/org/apache/carbondata/common/Strings.java
@@ -21,7 +21,7 @@ import java.util.Objects;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 
-@InterfaceAudience.Developer
+@InterfaceAudience.Internal
 public class Strings {
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e0591671/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
index fa9729d..8d214ff 100644
--- a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
+++ b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceAudience.java
@@ -25,10 +25,10 @@ import java.lang.annotation.RetentionPolicy;
  * This annotation is ported and modified from Apache Hadoop project.
  *
  * Annotation to inform users of a package, class or method's intended audience.
- * Currently the audience can be {@link User}, {@link Developer}
+ * Currently the audience can be {@link User}, {@link Developer}, {@link Internal}
  *
  * Public classes that are not marked with this annotation must be
- * considered by default as {@link Developer}.</li>
+ * considered by default as {@link Internal}.</li>
  *
  * External applications must only use classes that are marked {@link User}.
  *
@@ -47,12 +47,21 @@ public class InterfaceAudience {
   public @interface User { }
 
   /**
-   * Intended only for developers to extend interface for CarbonData project
-   * For example, new Datamap implementations.
+   * Intended for developers to develop extension for Apache CarbonData project
+   * For example, "Index DataMap" to add a new index implementation, etc
    */
   @Documented
   @Retention(RetentionPolicy.RUNTIME)
-  public @interface Developer { }
+  public @interface Developer {
+    String[] value();
+  }
+
+  /**
+   * Intended only for internal usage within Apache CarbonData project.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  public @interface Internal { }
 
   private InterfaceAudience() { } // Audience can't exist on its own
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e0591671/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
index b8e5e52..5435028 100644
--- a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
+++ b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java
@@ -36,7 +36,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience.*;
  *
  * <ul><li>All classes that are annotated with {@link User} or
  * {@link Developer} must have InterfaceStability annotation. </li>
- * <li>Classes that are {@link Private} are to be considered unstable unless
+ * <li>Classes that are {@link Internal} are to be considered unstable unless
  * a different InterfaceStability annotation states otherwise.</li>
  * <li>Incompatible changes must not be made to classes marked as stable.</li>
  * </ul>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e0591671/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
index fbb93b6..99684ad 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.conf.Configuration;
 /**
  * Builder for {@link CarbonLoadModel}
  */
-@InterfaceAudience.Developer
+@InterfaceAudience.Internal
 public class CarbonLoadModelBuilder {
 
   private CarbonTable table;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e0591671/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
index 8ec93a9..bd942ca 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
@@ -40,7 +40,7 @@ import org.apache.hadoop.conf.Configuration;
 /**
  * Provide utilities to populate loading options
  */
-@InterfaceAudience.Developer
+@InterfaceAudience.Internal
 public class LoadOption {
 
   private static LogService LOG = LogServiceFactory.getLogService(LoadOption.class.getName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/e0591671/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
index daea733..dc5696a 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
@@ -22,7 +22,6 @@ import java.util.Random;
 import java.util.UUID;
 
 import org.apache.carbondata.common.annotations.InterfaceAudience;
-import org.apache.carbondata.common.annotations.InterfaceStability;
 import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
 import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
@@ -40,8 +39,7 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 /**
  * Implementation to write rows in CSV format to carbondata file.
  */
-@InterfaceAudience.Developer
-@InterfaceStability.Unstable
+@InterfaceAudience.Internal
 class CSVCarbonWriter extends CarbonWriter {
 
   private RecordWriter<NullWritable, StringArrayWritable> recordWriter;


[17/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
deleted file mode 100644
index 6629d31..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datastore.chunk.impl;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
-import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-
-/**
- * This class is gives access to fixed length dimension data chunk store
- */
-public class FixedLengthDimensionDataChunk extends AbstractDimensionDataChunk {
-
-  /**
-   * Constructor
-   *
-   * @param dataChunk            data chunk
-   * @param invertedIndex        inverted index
-   * @param invertedIndexReverse reverse inverted index
-   * @param numberOfRows         number of rows
-   * @param columnValueSize      size of each column value
-   */
-  public FixedLengthDimensionDataChunk(byte[] dataChunk, int[] invertedIndex,
-      int[] invertedIndexReverse, int numberOfRows, int columnValueSize) {
-    long totalSize = null != invertedIndex ?
-        dataChunk.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) :
-        dataChunk.length;
-    dataChunkStore = DimensionChunkStoreFactory.INSTANCE
-        .getDimensionChunkStore(columnValueSize, null != invertedIndex, numberOfRows, totalSize,
-            DimensionStoreType.FIXEDLENGTH);
-    dataChunkStore.putArray(invertedIndex, invertedIndexReverse, dataChunk);
-  }
-
-  /**
-   * Below method will be used to fill the data based on offset and row id
-   *
-   * @param data             data to filed
-   * @param offset           offset from which data need to be filed
-   * @param index            row id of the chunk
-   * @param keyStructureInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int index,
-      KeyStructureInfo keyStructureInfo) {
-    dataChunkStore.fillRow(index, data, offset);
-    return dataChunkStore.getColumnValueSize();
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo
-   * @return
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo) {
-    row[columnIndex] = dataChunkStore.getSurrogate(rowId);
-    return columnIndex + 1;
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo) {
-    ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = columnVectorInfo.size + offset;
-    CarbonColumnVector vector = columnVectorInfo.vector;
-    for (int j = offset; j < len; j++) {
-      int dict = dataChunkStore.getSurrogate(j);
-      if (columnVectorInfo.directDictionaryGenerator == null) {
-        vector.putInt(vectorOffset++, dict);
-      } else {
-        Object valueFromSurrogate =
-            columnVectorInfo.directDictionaryGenerator.getValueFromSurrogate(dict);
-        if (valueFromSurrogate == null) {
-          vector.putNull(vectorOffset++);
-        } else {
-          DataType dataType = columnVectorInfo.directDictionaryGenerator.getReturnType();
-          if (dataType == DataTypes.INT) {
-            vector.putInt(vectorOffset++, (int) valueFromSurrogate);
-          } else if (dataType == DataTypes.LONG) {
-            vector.putLong(vectorOffset++, (long) valueFromSurrogate);
-          } else {
-            throw new IllegalArgumentException("unsupported data type: " +
-                columnVectorInfo.directDictionaryGenerator.getReturnType());
-          }
-        }
-      }
-    }
-    return column + 1;
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param rowMapping
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo,
-      int column, KeyStructureInfo restructuringInfo) {
-    ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = columnVectorInfo.size + offset;
-    CarbonColumnVector vector = columnVectorInfo.vector;
-    for (int j = offset; j < len; j++) {
-      int dict = dataChunkStore.getSurrogate(rowMapping[j]);
-      if (columnVectorInfo.directDictionaryGenerator == null) {
-        vector.putInt(vectorOffset++, dict);
-      } else {
-        Object valueFromSurrogate =
-            columnVectorInfo.directDictionaryGenerator.getValueFromSurrogate(dict);
-        if (valueFromSurrogate == null) {
-          vector.putNull(vectorOffset++);
-        } else {
-          DataType dataType = columnVectorInfo.directDictionaryGenerator.getReturnType();
-          if (dataType == DataTypes.INT) {
-            vector.putInt(vectorOffset++, (int) valueFromSurrogate);
-          } else if (dataType == DataTypes.LONG) {
-            vector.putLong(vectorOffset++, (long) valueFromSurrogate);
-          } else {
-            throw new IllegalArgumentException("unsupported data type: " +
-                columnVectorInfo.directDictionaryGenerator.getReturnType());
-          }
-        }
-      }
-    }
-    return column + 1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java
index fa0777b..5e8618b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/MeasureRawColumnChunk.java
@@ -19,7 +19,7 @@ package org.apache.carbondata.core.datastore.chunk.impl;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.AbstractRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReader;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -28,7 +28,7 @@ import org.apache.carbondata.core.memory.MemoryException;
 /**
  * Contains raw measure data
  * 1. The read uncompressed raw data of column chunk with all pages is stored in this instance.
- * 2. The raw data can be converted to processed chunk using convertToColumnPage method
+ * 2. The raw data can be converted to processed chunk using decodeColumnPage method
  *  by specifying page number.
  */
 public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
@@ -37,7 +37,7 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
 
   private MeasureColumnChunkReader chunkReader;
 
-  private FileHolder fileReader;
+  private FileReader fileReader;
 
   public MeasureRawColumnChunk(int columnIndex, ByteBuffer rawData, long offSet, int length,
       MeasureColumnChunkReader chunkReader) {
@@ -48,14 +48,14 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
   /**
    * Convert all raw data with all pages to processed ColumnPage
    */
-  public ColumnPage[] convertToColumnPage() {
+  public ColumnPage[] decodeAllColumnPages() {
     if (columnPages == null) {
       columnPages = new ColumnPage[pagesCount];
     }
     for (int i = 0; i < pagesCount; i++) {
       try {
         if (columnPages[i] == null) {
-          columnPages[i] = chunkReader.convertToColumnPage(this, i);
+          columnPages[i] = chunkReader.decodeColumnPage(this, i);
         }
       } catch (Exception e) {
         throw new RuntimeException(e);
@@ -68,21 +68,21 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
   /**
    * Convert raw data with specified `columnIndex` processed to ColumnPage
    */
-  public ColumnPage convertToColumnPage(int columnIndex) {
-    assert columnIndex < pagesCount;
+  public ColumnPage decodeColumnPage(int pageNumber) {
+    assert pageNumber < pagesCount;
     if (columnPages == null) {
       columnPages = new ColumnPage[pagesCount];
     }
 
     try {
-      if (columnPages[columnIndex] == null) {
-        columnPages[columnIndex] = chunkReader.convertToColumnPage(this, columnIndex);
+      if (columnPages[pageNumber] == null) {
+        columnPages[pageNumber] = chunkReader.decodeColumnPage(this, pageNumber);
       }
     } catch (IOException | MemoryException e) {
       throw new RuntimeException(e);
     }
 
-    return columnPages[columnIndex];
+    return columnPages[pageNumber];
   }
 
   /**
@@ -95,7 +95,7 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
     assert index < pagesCount;
 
     try {
-      return chunkReader.convertToColumnPage(this, index);
+      return chunkReader.decodeColumnPage(this, index);
     } catch (IOException | MemoryException e) {
       throw new RuntimeException(e);
     }
@@ -111,11 +111,11 @@ public class MeasureRawColumnChunk extends AbstractRawColumnChunk {
     }
   }
 
-  public void setFileReader(FileHolder fileReader) {
+  public void setFileReader(FileReader fileReader) {
     this.fileReader = fileReader;
   }
 
-  public FileHolder getFileReader() {
+  public FileReader getFileReader() {
     return fileReader;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java
new file mode 100644
index 0000000..d03b2de
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionColumnPage.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.datastore.chunk.impl;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
+import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+
+/**
+ * This class is gives access to variable length dimension data chunk store
+ */
+public class VariableLengthDimensionColumnPage extends AbstractDimensionColumnPage {
+
+  /**
+   * Constructor for this class
+   * @param dataChunks
+   * @param invertedIndex
+   * @param invertedIndexReverse
+   * @param numberOfRows
+   */
+  public VariableLengthDimensionColumnPage(byte[] dataChunks, int[] invertedIndex,
+      int[] invertedIndexReverse, int numberOfRows) {
+    long totalSize = null != invertedIndex ?
+        (dataChunks.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) + (
+            numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE)) :
+        (dataChunks.length + (numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE));
+    dataChunkStore = DimensionChunkStoreFactory.INSTANCE
+        .getDimensionChunkStore(0, null != invertedIndex, numberOfRows, totalSize,
+            DimensionStoreType.VARIABLELENGTH);
+    dataChunkStore.putArray(invertedIndex, invertedIndexReverse, dataChunks);
+  }
+
+  /**
+   * Below method will be used to fill the data based on offset and row id
+   *
+   * @param rowId             row id of the chunk
+   * @param offset            offset from which data need to be filed
+   * @param data              data to filed
+   * @param restructuringInfo define the structure of the key
+   * @return how many bytes was copied
+   */
+  @Override public int fillRawData(int rowId, int offset, byte[] data,
+      KeyStructureInfo restructuringInfo) {
+    // no required in this case because this column chunk is not the part if
+    // mdkey
+    return 0;
+  }
+
+  /**
+   * Converts to column dictionary integer value
+   *
+   * @param rowId
+   * @param chunkIndex
+   * @param outputSurrogateKey
+   * @param restructuringInfo
+   * @return
+   */
+  @Override public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
+      KeyStructureInfo restructuringInfo) {
+    return chunkIndex + 1;
+  }
+
+  /**
+   * @return whether column is dictionary column or not
+   */
+  @Override public boolean isNoDicitionaryColumn() {
+    return true;
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
+      KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    CarbonColumnVector vector = columnVectorInfo.vector;
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = offset + columnVectorInfo.size;
+    for (int i = offset; i < len; i++) {
+      // Considering only String case now as we support only
+      // string in no dictionary case at present.
+      dataChunkStore.fillRow(i, vector, vectorOffset++);
+    }
+    return chunkIndex + 1;
+  }
+
+  /**
+   * Fill the data to vector
+   *
+   * @param filteredRowId
+   * @param vectorInfo
+   * @param chunkIndex
+   * @param restructuringInfo
+   * @return next column index
+   */
+  @Override public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo,
+      int chunkIndex, KeyStructureInfo restructuringInfo) {
+    ColumnVectorInfo columnVectorInfo = vectorInfo[chunkIndex];
+    CarbonColumnVector vector = columnVectorInfo.vector;
+    int offset = columnVectorInfo.offset;
+    int vectorOffset = columnVectorInfo.vectorOffset;
+    int len = offset + columnVectorInfo.size;
+    for (int i = offset; i < len; i++) {
+      // Considering only String case now as we support only
+      // string in no dictionary case at present.
+      dataChunkStore.fillRow(filteredRowId[i], vector, vectorOffset++);
+    }
+    return chunkIndex + 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
deleted file mode 100644
index 6c47bf5..0000000
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.datastore.chunk.impl;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory;
-import org.apache.carbondata.core.datastore.chunk.store.DimensionChunkStoreFactory.DimensionStoreType;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-
-/**
- * This class is gives access to variable length dimension data chunk store
- */
-public class VariableLengthDimensionDataChunk extends AbstractDimensionDataChunk {
-
-  /**
-   * Constructor for this class
-   * @param dataChunks
-   * @param invertedIndex
-   * @param invertedIndexReverse
-   * @param numberOfRows
-   */
-  public VariableLengthDimensionDataChunk(byte[] dataChunks, int[] invertedIndex,
-      int[] invertedIndexReverse, int numberOfRows) {
-    long totalSize = null != invertedIndex ?
-        (dataChunks.length + (2 * numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE) + (
-            numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE)) :
-        (dataChunks.length + (numberOfRows * CarbonCommonConstants.INT_SIZE_IN_BYTE));
-    dataChunkStore = DimensionChunkStoreFactory.INSTANCE
-        .getDimensionChunkStore(0, null != invertedIndex, numberOfRows, totalSize,
-            DimensionStoreType.VARIABLELENGTH);
-    dataChunkStore.putArray(invertedIndex, invertedIndexReverse, dataChunks);
-  }
-
-  /**
-   * Below method will be used to fill the data based on offset and row id
-   *
-   * @param data              data to filed
-   * @param offset            offset from which data need to be filed
-   * @param index             row id of the chunk
-   * @param restructuringInfo define the structure of the key
-   * @return how many bytes was copied
-   */
-  @Override public int fillChunkData(byte[] data, int offset, int index,
-      KeyStructureInfo restructuringInfo) {
-    // no required in this case because this column chunk is not the part if
-    // mdkey
-    return 0;
-  }
-
-  /**
-   * Converts to column dictionary integer value
-   *
-   * @param rowId
-   * @param columnIndex
-   * @param row
-   * @param restructuringInfo
-   * @return
-   */
-  @Override public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
-      KeyStructureInfo restructuringInfo) {
-    return columnIndex + 1;
-  }
-
-  /**
-   * @return whether column is dictionary column or not
-   */
-  @Override public boolean isNoDicitionaryColumn() {
-    return true;
-  }
-
-  /**
-   * @return length of each column
-   */
-  @Override public int getColumnValueSize() {
-    return -1;
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
-      KeyStructureInfo restructuringInfo) {
-    ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-    CarbonColumnVector vector = columnVectorInfo.vector;
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = offset + columnVectorInfo.size;
-    for (int i = offset; i < len; i++) {
-      // Considering only String case now as we support only
-      // string in no dictionary case at present.
-      dataChunkStore.fillRow(i, vector, vectorOffset++);
-    }
-    return column + 1;
-  }
-
-  /**
-   * Fill the data to vector
-   *
-   * @param rowMapping
-   * @param vectorInfo
-   * @param column
-   * @param restructuringInfo
-   * @return next column index
-   */
-  @Override public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo,
-      int column, KeyStructureInfo restructuringInfo) {
-    ColumnVectorInfo columnVectorInfo = vectorInfo[column];
-    CarbonColumnVector vector = columnVectorInfo.vector;
-    int offset = columnVectorInfo.offset;
-    int vectorOffset = columnVectorInfo.vectorOffset;
-    int len = offset + columnVectorInfo.size;
-    for (int i = offset; i < len; i++) {
-      // Considering only String case now as we support only
-      // string in no dictionary case at present.
-      dataChunkStore.fillRow(rowMapping[i], vector, vectorOffset++);
-    }
-    return column + 1;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java
index 7b5b9c8..fd81973 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/DimensionColumnChunkReader.java
@@ -18,8 +18,8 @@ package org.apache.carbondata.core.datastore.chunk.reader;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.memory.MemoryException;
 
@@ -34,20 +34,20 @@ public interface DimensionColumnChunkReader {
    * Below method will be used to read the chunk based on block indexes
    *
    * @param fileReader   file reader to read the blocks from file
-   * @param blockletIndexes blocklets to be read
+   * @param columnIndexRange blocklets to be read
    * @return dimension column chunks
    */
-  DimensionRawColumnChunk[] readRawDimensionChunks(FileHolder fileReader, int[][] blockletIndexes)
+  DimensionRawColumnChunk[] readRawDimensionChunks(FileReader fileReader, int[][] columnIndexRange)
       throws IOException;
 
   /**
    * Below method will be used to read the chunk based on block index
    *
    * @param fileReader file reader to read the blocks from file
-   * @param blockletIndex block to be read
+   * @param columnIndex column to be read
    * @return dimension column chunk
    */
-  DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int blockletIndex)
+  DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, int columnIndex)
       throws IOException;
 
   /**
@@ -58,6 +58,6 @@ public interface DimensionColumnChunkReader {
    * @return
    * @throws IOException
    */
-  DimensionColumnDataChunk convertToDimensionChunk(DimensionRawColumnChunk dimensionRawColumnChunk,
+  DimensionColumnPage decodeColumnPage(DimensionRawColumnChunk dimensionRawColumnChunk,
       int pageNumber) throws IOException, MemoryException;
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java
index 02dc6a2..bf76025 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/MeasureColumnChunkReader.java
@@ -18,7 +18,7 @@ package org.apache.carbondata.core.datastore.chunk.reader;
 
 import java.io.IOException;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.memory.MemoryException;
@@ -32,20 +32,20 @@ public interface MeasureColumnChunkReader {
    * Method to read the blocks data based on block indexes
    *
    * @param fileReader   file reader to read the blocks
-   * @param blockIndexes blocks to be read
+   * @param columnIndexRange blocks to be read
    * @return measure data chunks
    */
-  MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader, int[][] blockIndexes)
+  MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader, int[][] columnIndexRange)
       throws IOException;
 
   /**
    * Method to read the blocks data based on block index
    *
    * @param fileReader file reader to read the blocks
-   * @param blockIndex block to be read
+   * @param columnIndex block to be read
    * @return measure data chunk
    */
-  MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int blockIndex)
+  MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException;
 
   /**
@@ -55,7 +55,7 @@ public interface MeasureColumnChunkReader {
    * @return
    * @throws IOException
    */
-  ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
+  ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
       int pageNumber) throws IOException, MemoryException;
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
index f083612..60950c9 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/AbstractChunkReaderV2V3Format.java
@@ -19,7 +19,7 @@ package org.apache.carbondata.core.datastore.chunk.reader.dimension;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.format.Encoding;
@@ -55,43 +55,43 @@ public abstract class AbstractChunkReaderV2V3Format extends AbstractChunkReader
    * For last column read is separately and process
    *
    * @param fileReader      file reader to read the blocks from file
-   * @param blockletIndexes blocks range to be read
+   * @param columnIndexRange column index range to be read
    * @return dimension column chunks
    */
-  @Override public DimensionRawColumnChunk[] readRawDimensionChunks(final FileHolder fileReader,
-      final int[][] blockletIndexes) throws IOException {
+  @Override public DimensionRawColumnChunk[] readRawDimensionChunks(final FileReader fileReader,
+      final int[][] columnIndexRange) throws IOException {
     // read the column chunk based on block index and add
     DimensionRawColumnChunk[] dataChunks =
         new DimensionRawColumnChunk[dimensionChunksOffset.size()];
     // if blocklet index is empty then return empry data chunk
-    if (blockletIndexes.length == 0) {
+    if (columnIndexRange.length == 0) {
       return dataChunks;
     }
     DimensionRawColumnChunk[] groupChunk = null;
     int index = 0;
     // iterate till block indexes -1 as block index will be in sorted order, so to avoid
     // the last column reading in group
-    for (int i = 0; i < blockletIndexes.length - 1; i++) {
+    for (int i = 0; i < columnIndexRange.length - 1; i++) {
       index = 0;
       groupChunk =
-          readRawDimensionChunksInGroup(fileReader, blockletIndexes[i][0], blockletIndexes[i][1]);
-      for (int j = blockletIndexes[i][0]; j <= blockletIndexes[i][1]; j++) {
+          readRawDimensionChunksInGroup(fileReader, columnIndexRange[i][0], columnIndexRange[i][1]);
+      for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) {
         dataChunks[j] = groupChunk[index++];
       }
     }
     // check last index is present in block index, if it is present then read separately
-    if (blockletIndexes[blockletIndexes.length - 1][0] == dimensionChunksOffset.size() - 1) {
-      dataChunks[blockletIndexes[blockletIndexes.length - 1][0]] =
-          readRawDimensionChunk(fileReader, blockletIndexes[blockletIndexes.length - 1][0]);
+    if (columnIndexRange[columnIndexRange.length - 1][0] == dimensionChunksOffset.size() - 1) {
+      dataChunks[columnIndexRange[columnIndexRange.length - 1][0]] =
+          readRawDimensionChunk(fileReader, columnIndexRange[columnIndexRange.length - 1][0]);
     }
     // otherwise read the data in group
     else {
-      groupChunk =
-          readRawDimensionChunksInGroup(fileReader, blockletIndexes[blockletIndexes.length - 1][0],
-              blockletIndexes[blockletIndexes.length - 1][1]);
+      groupChunk = readRawDimensionChunksInGroup(
+          fileReader, columnIndexRange[columnIndexRange.length - 1][0],
+          columnIndexRange[columnIndexRange.length - 1][1]);
       index = 0;
-      for (int j = blockletIndexes[blockletIndexes.length - 1][0];
-           j <= blockletIndexes[blockletIndexes.length - 1][1]; j++) {
+      for (int j = columnIndexRange[columnIndexRange.length - 1][0];
+           j <= columnIndexRange[columnIndexRange.length - 1][1]; j++) {
         dataChunks[j] = groupChunk[index++];
       }
     }
@@ -109,7 +109,7 @@ public abstract class AbstractChunkReaderV2V3Format extends AbstractChunkReader
    * @return measure raw chunkArray
    * @throws IOException
    */
-  protected abstract DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
+  protected abstract DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
index 27a4d89..0dc1c1b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v1/CompressedDimensionChunkFileBasedReaderV1.java
@@ -20,12 +20,12 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReader;
 import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -61,14 +61,14 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
    * Below method will be used to read the raw chunk based on block indexes
    *
    * @param fileReader   file reader to read the blocks from file
-   * @param blockletIndexes blocks to be read
+   * @param columnIndexRange blocks to be read
    * @return dimension column chunks
    */
-  @Override public DimensionRawColumnChunk[] readRawDimensionChunks(FileHolder fileReader,
-      int[][] blockletIndexes) throws IOException {
+  @Override public DimensionRawColumnChunk[] readRawDimensionChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     DimensionRawColumnChunk[] dataChunks = new DimensionRawColumnChunk[dimensionColumnChunk.size()];
-    for (int i = 0; i < blockletIndexes.length; i++) {
-      for (int j = blockletIndexes[i][0]; j <= blockletIndexes[i][1]; j++) {
+    for (int i = 0; i < columnIndexRange.length; i++) {
+      for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) {
         dataChunks[j] = readRawDimensionChunk(fileReader, j);
       }
     }
@@ -82,7 +82,7 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
    * @param columnIndex column to be read
    * @return dimension column chunk
    */
-  @Override public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
+  @Override public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader,
       int columnIndex) throws IOException {
     DataChunk dataChunk = dimensionColumnChunk.get(columnIndex);
     ByteBuffer buffer = null;
@@ -92,20 +92,20 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
     }
     DimensionRawColumnChunk rawColumnChunk = new DimensionRawColumnChunk(columnIndex, buffer, 0,
         dataChunk.getDataPageLength(), this);
-    rawColumnChunk.setFileHolder(fileReader);
+    rawColumnChunk.setFileReader(fileReader);
     rawColumnChunk.setPagesCount(1);
     rawColumnChunk.setRowCount(new int[] { numberOfRows });
     return rawColumnChunk;
   }
 
-  @Override public DimensionColumnDataChunk convertToDimensionChunk(
+  @Override public DimensionColumnPage decodeColumnPage(
       DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException {
     int blockIndex = dimensionRawColumnChunk.getColumnIndex();
     byte[] dataPage = null;
     int[] invertedIndexes = null;
     int[] invertedIndexesReverse = null;
     int[] rlePage = null;
-    FileHolder fileReader = dimensionRawColumnChunk.getFileReader();
+    FileReader fileReader = dimensionRawColumnChunk.getFileReader();
 
     ByteBuffer rawData = dimensionRawColumnChunk.getRawData();
     dataPage = COMPRESSOR.unCompressByte(rawData.array(), (int) dimensionRawColumnChunk.getOffSet(),
@@ -145,23 +145,23 @@ public class CompressedDimensionChunkFileBasedReaderV1 extends AbstractChunkRead
       rlePage = null;
     }
     // fill chunk attributes
-    DimensionColumnDataChunk columnDataChunk = null;
+    DimensionColumnPage columnDataChunk = null;
     if (dataChunk.isRowMajor()) {
       // to store fixed length column chunk values
-      columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, eachColumnValueSize[blockIndex],
-          numberOfRows);
+      columnDataChunk = new ColumnGroupDimensionColumnPage(
+          dataPage, eachColumnValueSize[blockIndex], numberOfRows);
     }
     // if no dictionary column then first create a no dictionary column chunk
     // and set to data chunk instance
     else if (!CarbonUtil
         .hasEncoding(dataChunk.getEncodingList(), Encoding.DICTIONARY)) {
       columnDataChunk =
-          new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows);
     } else {
       // to store fixed length column chunk values
       columnDataChunk =
-          new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows, eachColumnValueSize[blockIndex]);
     }
     return columnDataChunk;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
index b43f89c..31fa819 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v2/CompressedDimensionChunkFileBasedReaderV2.java
@@ -19,12 +19,12 @@ package org.apache.carbondata.core.datastore.chunk.reader.dimension.v2;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.ColumnGroupDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReaderV2V3Format;
 import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -56,7 +56,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
    * @param columnIndex   column to be read
    * @return dimension column chunk
    */
-  public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader, int columnIndex)
+  public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     int length = 0;
     if (dimensionChunksOffset.size() - 1 == columnIndex) {
@@ -73,7 +73,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
     }
     DimensionRawColumnChunk rawColumnChunk =
         new DimensionRawColumnChunk(columnIndex, buffer, 0, length, this);
-    rawColumnChunk.setFileHolder(fileReader);
+    rawColumnChunk.setFileReader(fileReader);
     rawColumnChunk.setPagesCount(1);
     rawColumnChunk.setRowCount(new int[] { numberOfRows });
     return rawColumnChunk;
@@ -90,7 +90,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
    * @return measure raw chunkArray
    * @throws IOException
    */
-  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
+  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
     long currentDimensionOffset = dimensionChunksOffset.get(startColumnBlockletIndex);
     ByteBuffer buffer = null;
@@ -106,7 +106,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
       int currentLength = (int) (dimensionChunksOffset.get(i + 1) - dimensionChunksOffset.get(i));
       dataChunks[index] =
           new DimensionRawColumnChunk(i, buffer, runningLength, currentLength, this);
-      dataChunks[index].setFileHolder(fileReader);
+      dataChunks[index].setFileReader(fileReader);
       dataChunks[index].setPagesCount(1);
       dataChunks[index].setRowCount(new int[] { numberOfRows });
       runningLength += currentLength;
@@ -115,7 +115,7 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
     return dataChunks;
   }
 
-  public DimensionColumnDataChunk convertToDimensionChunk(
+  public DimensionColumnPage decodeColumnPage(
       DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber) throws IOException {
     byte[] dataPage = null;
     int[] invertedIndexes = null;
@@ -169,23 +169,23 @@ public class CompressedDimensionChunkFileBasedReaderV2 extends AbstractChunkRead
       dataPage = UnBlockIndexer.uncompressData(dataPage, rlePage, eachColumnValueSize[blockIndex]);
     }
     // fill chunk attributes
-    DimensionColumnDataChunk columnDataChunk = null;
+    DimensionColumnPage columnDataChunk = null;
 
     if (dimensionColumnChunk.isRowMajor()) {
       // to store fixed length column chunk values
-      columnDataChunk = new ColumnGroupDimensionDataChunk(dataPage, eachColumnValueSize[blockIndex],
-          numberOfRows);
+      columnDataChunk = new ColumnGroupDimensionColumnPage(
+          dataPage, eachColumnValueSize[blockIndex], numberOfRows);
     }
     // if no dictionary column then first create a no dictionary column chunk
     // and set to data chunk instance
     else if (!hasEncoding(dimensionColumnChunk.encoders, Encoding.DICTIONARY)) {
       columnDataChunk =
-          new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows);
     } else {
       // to store fixed length column chunk values
       columnDataChunk =
-          new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               numberOfRows, eachColumnValueSize[blockIndex]);
     }
     return columnDataChunk;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
index 1edfd09..60f0b67 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimChunkFileBasedPageLevelReaderV3.java
@@ -20,8 +20,8 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.metadata.blocklet.BlockletInfo;
@@ -70,7 +70,8 @@ public class CompressedDimChunkFileBasedPageLevelReaderV3
    * @param blockletColumnIndex blocklet index of the column in carbon data file
    * @return dimension raw chunk
    */
-  public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
+  @Override
+  public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader,
       int blockletColumnIndex) throws IOException {
     // get the current dimension offset
     long currentDimensionOffset = dimensionChunksOffset.get(blockletColumnIndex);
@@ -116,7 +117,7 @@ public class CompressedDimChunkFileBasedPageLevelReaderV3
    * @param endBlockletColumnIndex   blocklet index of the last dimension column
    * @ DimensionRawColumnChunk array
    */
-  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
+  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startBlockletColumnIndex, int endBlockletColumnIndex) throws IOException {
     // create raw chunk for each dimension column
     DimensionRawColumnChunk[] dimensionDataChunks =
@@ -136,7 +137,7 @@ public class CompressedDimChunkFileBasedPageLevelReaderV3
    * @param pageNumber              number
    * @return DimensionColumnDataChunk
    */
-  @Override public DimensionColumnDataChunk convertToDimensionChunk(
+  @Override public DimensionColumnPage decodeColumnPage(
       DimensionRawColumnChunk dimensionRawColumnChunk, int pageNumber)
       throws IOException, MemoryException {
     // data chunk of page

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
index 566e9b7..0fdc515 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/dimension/v3/CompressedDimensionChunkFileBasedReaderV3.java
@@ -20,11 +20,11 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.VariableLengthDimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.reader.dimension.AbstractChunkReaderV2V3Format;
 import org.apache.carbondata.core.datastore.chunk.store.ColumnPageWrapper;
 import org.apache.carbondata.core.datastore.columnar.UnBlockIndexer;
@@ -78,23 +78,23 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
    * 5. Create the raw chunk object and fill the details
    *
    * @param fileReader          reader for reading the column from carbon data file
-   * @param blockletColumnIndex blocklet index of the column in carbon data file
+   * @param columnIndex blocklet index of the column in carbon data file
    * @return dimension raw chunk
    */
-  public DimensionRawColumnChunk readRawDimensionChunk(FileHolder fileReader,
-      int blockletColumnIndex) throws IOException {
+  public DimensionRawColumnChunk readRawDimensionChunk(FileReader fileReader,
+      int columnIndex) throws IOException {
     // get the current dimension offset
-    long currentDimensionOffset = dimensionChunksOffset.get(blockletColumnIndex);
+    long currentDimensionOffset = dimensionChunksOffset.get(columnIndex);
     int length = 0;
     // to calculate the length of the data to be read
     // column other than last column we can subtract the offset of current column with
     // next column and get the total length.
     // but for last column we need to use lastDimensionOffset which is the end position
     // of the last dimension, we can subtract current dimension offset from lastDimesionOffset
-    if (dimensionChunksOffset.size() - 1 == blockletColumnIndex) {
+    if (dimensionChunksOffset.size() - 1 == columnIndex) {
       length = (int) (lastDimensionOffsets - currentDimensionOffset);
     } else {
-      length = (int) (dimensionChunksOffset.get(blockletColumnIndex + 1) - currentDimensionOffset);
+      length = (int) (dimensionChunksOffset.get(columnIndex + 1) - currentDimensionOffset);
     }
     ByteBuffer buffer = null;
     // read the data from carbon data file
@@ -103,15 +103,15 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     }
     // get the data chunk which will have all the details about the data pages
     DataChunk3 dataChunk = CarbonUtil.readDataChunk3(buffer, 0, length);
-    return getDimensionRawColumnChunk(fileReader, blockletColumnIndex, 0, length, buffer,
+    return getDimensionRawColumnChunk(fileReader, columnIndex, 0, length, buffer,
         dataChunk);
   }
 
-  protected DimensionRawColumnChunk getDimensionRawColumnChunk(FileHolder fileReader,
-      int blockletColumnIndex, long offset, int length, ByteBuffer buffer, DataChunk3 dataChunk) {
+  protected DimensionRawColumnChunk getDimensionRawColumnChunk(FileReader fileReader,
+      int columnIndex, long offset, int length, ByteBuffer buffer, DataChunk3 dataChunk) {
     // creating a raw chunks instance and filling all the details
     DimensionRawColumnChunk rawColumnChunk =
-        new DimensionRawColumnChunk(blockletColumnIndex, buffer, offset, length, this);
+        new DimensionRawColumnChunk(columnIndex, buffer, offset, length, this);
     int numberOfPages = dataChunk.getPage_length().size();
     byte[][] maxValueOfEachPage = new byte[numberOfPages][];
     byte[][] minValueOfEachPage = new byte[numberOfPages][];
@@ -124,7 +124,7 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
       eachPageLength[i] = dataChunk.getData_chunk_list().get(i).getNumberOfRowsInpage();
     }
     rawColumnChunk.setDataChunkV3(dataChunk);
-    rawColumnChunk.setFileHolder(fileReader);
+    rawColumnChunk.setFileReader(fileReader);
     rawColumnChunk.setPagesCount(dataChunk.getPage_length().size());
     rawColumnChunk.setMaxValues(maxValueOfEachPage);
     rawColumnChunk.setMinValues(minValueOfEachPage);
@@ -153,7 +153,7 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
    *        blocklet index of the last dimension column
    * @ DimensionRawColumnChunk array
    */
-  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileHolder fileReader,
+  protected DimensionRawColumnChunk[] readRawDimensionChunksInGroup(FileReader fileReader,
       int startBlockletColumnIndex, int endBlockletColumnIndex) throws IOException {
     // to calculate the length of the data to be read
     // column we can subtract the offset of start column offset with
@@ -188,9 +188,9 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
    *
    * @param rawColumnPage dimension raw chunk
    * @param pageNumber              number
-   * @return DimensionColumnDataChunk
+   * @return DimensionColumnPage
    */
-  @Override public DimensionColumnDataChunk convertToDimensionChunk(
+  @Override public DimensionColumnPage decodeColumnPage(
       DimensionRawColumnChunk rawColumnPage, int pageNumber) throws IOException, MemoryException {
     // data chunk of blocklet column
     DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3();
@@ -228,20 +228,19 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
     return false;
   }
 
-  protected DimensionColumnDataChunk decodeDimension(DimensionRawColumnChunk rawColumnPage,
+  protected DimensionColumnPage decodeDimension(DimensionRawColumnChunk rawColumnPage,
       ByteBuffer pageData, DataChunk2 pageMetadata, int offset)
       throws IOException, MemoryException {
     if (isEncodedWithMeta(pageMetadata)) {
       ColumnPage decodedPage = decodeDimensionByMeta(pageMetadata, pageData, offset);
-      return new ColumnPageWrapper(decodedPage,
-          eachColumnValueSize[rawColumnPage.getColumnIndex()]);
+      return new ColumnPageWrapper(decodedPage);
     } else {
       // following code is for backward compatibility
       return decodeDimensionLegacy(rawColumnPage, pageData, pageMetadata, offset);
     }
   }
 
-  private DimensionColumnDataChunk decodeDimensionLegacy(DimensionRawColumnChunk rawColumnPage,
+  private DimensionColumnPage decodeDimensionLegacy(DimensionRawColumnChunk rawColumnPage,
       ByteBuffer pageData, DataChunk2 pageMetadata, int offset) {
     byte[] dataPage;
     int[] rlePage;
@@ -267,18 +266,18 @@ public class CompressedDimensionChunkFileBasedReaderV3 extends AbstractChunkRead
           eachColumnValueSize[rawColumnPage.getColumnIndex()]);
     }
 
-    DimensionColumnDataChunk columnDataChunk = null;
+    DimensionColumnPage columnDataChunk = null;
 
     // if no dictionary column then first create a no dictionary column chunk
     // and set to data chunk instance
     if (!hasEncoding(pageMetadata.encoders, Encoding.DICTIONARY)) {
       columnDataChunk =
-          new VariableLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new VariableLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               pageMetadata.getNumberOfRowsInpage());
     } else {
       // to store fixed length column chunk values
       columnDataChunk =
-          new FixedLengthDimensionDataChunk(dataPage, invertedIndexes, invertedIndexesReverse,
+          new FixedLengthDimensionColumnPage(dataPage, invertedIndexes, invertedIndexesReverse,
               pageMetadata.getNumberOfRowsInpage(),
               eachColumnValueSize[rawColumnPage.getColumnIndex()]);
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java
index 2239a2b..a3ed339 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/AbstractMeasureChunkReaderV2V3Format.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.util.BitSet;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.compression.Compressor;
 import org.apache.carbondata.core.datastore.compression.CompressorFactory;
@@ -56,36 +56,40 @@ public abstract class AbstractMeasureChunkReaderV2V3Format extends AbstractMeasu
    * separately and process
    *
    * @param fileReader   file reader to read the blocks from file
-   * @param blockIndexes blocks range to be read
+   * @param columnIndexRange blocks range to be read, columnIndexGroup[i] is one group, inside the
+   *                         group, columnIndexGroup[i][0] is start column index,
+   *                         and columnIndexGroup[i][1] is end column index
    * @return measure column chunks
    * @throws IOException
    */
-  public MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader, int[][] blockIndexes)
-      throws IOException {
+  public MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     // read the column chunk based on block index and add
     MeasureRawColumnChunk[] dataChunks =
         new MeasureRawColumnChunk[measureColumnChunkOffsets.size()];
-    if (blockIndexes.length == 0) {
+    if (columnIndexRange.length == 0) {
       return dataChunks;
     }
     MeasureRawColumnChunk[] groupChunk = null;
     int index = 0;
-    for (int i = 0; i < blockIndexes.length - 1; i++) {
+    for (int i = 0; i < columnIndexRange.length - 1; i++) {
       index = 0;
-      groupChunk = readRawMeasureChunksInGroup(fileReader, blockIndexes[i][0], blockIndexes[i][1]);
-      for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) {
+      groupChunk = readRawMeasureChunksInGroup(
+          fileReader, columnIndexRange[i][0], columnIndexRange[i][1]);
+      for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) {
         dataChunks[j] = groupChunk[index++];
       }
     }
-    if (blockIndexes[blockIndexes.length - 1][0] == measureColumnChunkOffsets.size() - 1) {
-      dataChunks[blockIndexes[blockIndexes.length - 1][0]] =
-          readRawMeasureChunk(fileReader, blockIndexes[blockIndexes.length - 1][0]);
+    if (columnIndexRange[columnIndexRange.length - 1][0] == measureColumnChunkOffsets.size() - 1) {
+      dataChunks[columnIndexRange[columnIndexRange.length - 1][0]] =
+          readRawMeasureChunk(fileReader, columnIndexRange[columnIndexRange.length - 1][0]);
     } else {
-      groupChunk = readRawMeasureChunksInGroup(fileReader, blockIndexes[blockIndexes.length - 1][0],
-          blockIndexes[blockIndexes.length - 1][1]);
+      groupChunk = readRawMeasureChunksInGroup(
+          fileReader, columnIndexRange[columnIndexRange.length - 1][0],
+          columnIndexRange[columnIndexRange.length - 1][1]);
       index = 0;
-      for (int j = blockIndexes[blockIndexes.length - 1][0];
-           j <= blockIndexes[blockIndexes.length - 1][1]; j++) {
+      for (int j = columnIndexRange[columnIndexRange.length - 1][0];
+           j <= columnIndexRange[columnIndexRange.length - 1][1]; j++) {
         dataChunks[j] = groupChunk[index++];
       }
     }
@@ -112,12 +116,12 @@ public abstract class AbstractMeasureChunkReaderV2V3Format extends AbstractMeasu
    * data from
    *
    * @param fileReader               file reader to read the data
-   * @param startColumnBlockletIndex first column blocklet index to be read
-   * @param endColumnBlockletIndex   end column blocklet index to be read
+   * @param startColumnIndex first column index to be read
+   * @param endColumnIndex   end column index to be read
    * @return measure raw chunkArray
    * @throws IOException
    */
-  protected abstract MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
-      int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException;
+  protected abstract MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader,
+      int startColumnIndex, int endColumnIndex) throws IOException;
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
index ae55375..f0c1b75 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v1/CompressedMeasureChunkFileBasedReaderV1.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReader;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -56,14 +56,14 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
    * Method to read the blocks data based on block indexes
    *
    * @param fileReader   file reader to read the blocks
-   * @param blockIndexes blocks to be read
+   * @param columnIndexRange blocks to be read
    * @return measure data chunks
    */
-  @Override public MeasureRawColumnChunk[] readRawMeasureChunks(FileHolder fileReader,
-      int[][] blockIndexes) throws IOException {
+  @Override public MeasureRawColumnChunk[] readRawMeasureChunks(FileReader fileReader,
+      int[][] columnIndexRange) throws IOException {
     MeasureRawColumnChunk[] datChunk = new MeasureRawColumnChunk[measureColumnChunks.size()];
-    for (int i = 0; i < blockIndexes.length; i++) {
-      for (int j = blockIndexes[i][0]; j <= blockIndexes[i][1]; j++) {
+    for (int i = 0; i < columnIndexRange.length; i++) {
+      for (int j = columnIndexRange[i][0]; j <= columnIndexRange[i][1]; j++) {
         datChunk[j] = readRawMeasureChunk(fileReader, j);
       }
     }
@@ -77,7 +77,7 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
    * @param columnIndex column to be read
    * @return measure data chunk
    */
-  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int columnIndex)
+  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     DataChunk dataChunk = measureColumnChunks.get(columnIndex);
     ByteBuffer buffer = fileReader
@@ -91,7 +91,7 @@ public class CompressedMeasureChunkFileBasedReaderV1 extends AbstractMeasureChun
   }
 
   @Override
-  public ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
+  public ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
       int pageNumber) throws IOException, MemoryException {
     int blockIndex = measureRawColumnChunk.getColumnIndex();
     DataChunk dataChunk = measureColumnChunks.get(blockIndex);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
index d61f98a..04d6e2e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v2/CompressedMeasureChunkFileBasedReaderV2.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReaderV2V3Format;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -48,7 +48,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
   }
 
   @Override
-  public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader, int columnIndex)
+  public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader, int columnIndex)
       throws IOException {
     int dataLength = 0;
     if (measureColumnChunkOffsets.size() - 1 == columnIndex) {
@@ -83,30 +83,30 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
    * data from
    *
    * @param fileReader               file reader to read the data
-   * @param startColumnBlockletIndex first column blocklet index to be read
-   * @param endColumnBlockletIndex   end column blocklet index to be read
+   * @param startColumnIndex first column blocklet index to be read
+   * @param endColumnIndex   end column blocklet index to be read
    * @return measure raw chunkArray
    * @throws IOException
    */
-  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
-      int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
-    long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnBlockletIndex);
+  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader,
+      int startColumnIndex, int endColumnIndex) throws IOException {
+    long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnIndex);
     ByteBuffer buffer = null;
     synchronized (fileReader) {
       buffer = fileReader.readByteBuffer(filePath, currentMeasureOffset,
-          (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset));
+          (int) (measureColumnChunkOffsets.get(endColumnIndex + 1) - currentMeasureOffset));
     }
     MeasureRawColumnChunk[] dataChunks =
-        new MeasureRawColumnChunk[endColumnBlockletIndex - startColumnBlockletIndex + 1];
+        new MeasureRawColumnChunk[endColumnIndex - startColumnIndex + 1];
     int runningLength = 0;
     int index = 0;
-    for (int i = startColumnBlockletIndex; i <= endColumnBlockletIndex; i++) {
+    for (int i = startColumnIndex; i <= endColumnIndex; i++) {
       int currentLength =
           (int) (measureColumnChunkOffsets.get(i + 1) - measureColumnChunkOffsets.get(i));
       MeasureRawColumnChunk measureRawColumnChunk =
           new MeasureRawColumnChunk(i, buffer, runningLength, currentLength, this);
-      measureRawColumnChunk.setFileReader(fileReader);
       measureRawColumnChunk.setRowCount(new int[] { numberOfRows });
+      measureRawColumnChunk.setFileReader(fileReader);
       measureRawColumnChunk.setPagesCount(1);
       dataChunks[index] = measureRawColumnChunk;
       runningLength += currentLength;
@@ -115,7 +115,7 @@ public class CompressedMeasureChunkFileBasedReaderV2 extends AbstractMeasureChun
     return dataChunks;
   }
 
-  public ColumnPage convertToColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
+  public ColumnPage decodeColumnPage(MeasureRawColumnChunk measureRawColumnChunk,
       int pageNumber) throws IOException, MemoryException {
     int copyPoint = (int) measureRawColumnChunk.getOffSet();
     int blockIndex = measureRawColumnChunk.getColumnIndex();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
index 4f6987b..6dc02a3 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMeasureChunkFileBasedReaderV3.java
@@ -20,7 +20,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.reader.measure.AbstractMeasureChunkReaderV2V3Format;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -70,7 +70,7 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
    * @param columnIndex         column to be read
    * @return measure raw chunk
    */
-  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader,
+  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader,
       int columnIndex) throws IOException {
     int dataLength = 0;
     // to calculate the length of the data to be read
@@ -99,9 +99,8 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
         dataChunk);
   }
 
-  protected MeasureRawColumnChunk getMeasureRawColumnChunk(FileHolder fileReader,
-      int columnIndex, long offset, int dataLength, ByteBuffer buffer,
-      DataChunk3 dataChunk) {
+  MeasureRawColumnChunk getMeasureRawColumnChunk(FileReader fileReader, int columnIndex,
+      long offset, int dataLength, ByteBuffer buffer, DataChunk3 dataChunk) {
     // creating a raw chunks instance and filling all the details
     MeasureRawColumnChunk rawColumnChunk =
         new MeasureRawColumnChunk(columnIndex, buffer, offset, dataLength, this);
@@ -140,30 +139,30 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
    *
    * @param fileReader
    *        reader which will be used to read the measure columns data from file
-   * @param startColumnBlockletIndex
-   *        blocklet index of the first measure column
-   * @param endColumnBlockletIndex
-   *        blocklet index of the last measure column
+   * @param startColumnIndex
+   *        column index of the first measure column
+   * @param endColumnIndex
+   *        column index of the last measure column
    * @return MeasureRawColumnChunk array
    */
-  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
-      int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
+  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader,
+      int startColumnIndex, int endColumnIndex) throws IOException {
     // to calculate the length of the data to be read
     // column we can subtract the offset of start column offset with
     // end column+1 offset and get the total length.
-    long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnBlockletIndex);
+    long currentMeasureOffset = measureColumnChunkOffsets.get(startColumnIndex);
     ByteBuffer buffer = null;
     // read the data from carbon data file
     synchronized (fileReader) {
       buffer = fileReader.readByteBuffer(filePath, currentMeasureOffset,
-          (int) (measureColumnChunkOffsets.get(endColumnBlockletIndex + 1) - currentMeasureOffset));
+          (int) (measureColumnChunkOffsets.get(endColumnIndex + 1) - currentMeasureOffset));
     }
     // create raw chunk for each measure column
     MeasureRawColumnChunk[] measureDataChunk =
-        new MeasureRawColumnChunk[endColumnBlockletIndex - startColumnBlockletIndex + 1];
+        new MeasureRawColumnChunk[endColumnIndex - startColumnIndex + 1];
     int runningLength = 0;
     int index = 0;
-    for (int i = startColumnBlockletIndex; i <= endColumnBlockletIndex; i++) {
+    for (int i = startColumnIndex; i <= endColumnIndex; i++) {
       int currentLength =
           (int) (measureColumnChunkOffsets.get(i + 1) - measureColumnChunkOffsets.get(i));
       DataChunk3 dataChunk =
@@ -180,25 +179,25 @@ public class CompressedMeasureChunkFileBasedReaderV3 extends AbstractMeasureChun
   /**
    * Below method will be used to convert the compressed measure chunk raw data to actual data
    *
-   * @param rawColumnPage measure raw chunk
+   * @param rawColumnChunk measure raw chunk
    * @param pageNumber            number
-   * @return DimensionColumnDataChunk
+   * @return DimensionColumnPage
    */
   @Override
-  public ColumnPage convertToColumnPage(
-      MeasureRawColumnChunk rawColumnPage, int pageNumber)
+  public ColumnPage decodeColumnPage(
+      MeasureRawColumnChunk rawColumnChunk, int pageNumber)
       throws IOException, MemoryException {
     // data chunk of blocklet column
-    DataChunk3 dataChunk3 = rawColumnPage.getDataChunkV3();
+    DataChunk3 dataChunk3 = rawColumnChunk.getDataChunkV3();
     // data chunk of page
     DataChunk2 pageMetadata = dataChunk3.getData_chunk_list().get(pageNumber);
     // calculating the start point of data
     // as buffer can contain multiple column data, start point will be datachunkoffset +
     // data chunk length + page offset
-    int offset = (int) rawColumnPage.getOffSet() +
-        measureColumnChunkLength.get(rawColumnPage.getColumnIndex()) +
+    int offset = (int) rawColumnChunk.getOffSet() +
+        measureColumnChunkLength.get(rawColumnChunk.getColumnIndex()) +
         dataChunk3.getPage_offset().get(pageNumber);
-    ColumnPage decodedPage = decodeMeasure(pageMetadata, rawColumnPage.getRawData(), offset);
+    ColumnPage decodedPage = decodeMeasure(pageMetadata, rawColumnChunk.getRawData(), offset);
     decodedPage.setNullBits(getNullBitSet(pageMetadata.presence));
     return decodedPage;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java
index 31ff4c0..6b37575 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/reader/measure/v3/CompressedMsrChunkFileBasedPageLevelReaderV3.java
@@ -20,7 +20,7 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.memory.MemoryException;
@@ -66,7 +66,7 @@ public class CompressedMsrChunkFileBasedPageLevelReaderV3
    * @param blockletColumnIndex blocklet index of the column in carbon data file
    * @return measure raw chunk
    */
-  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileHolder fileReader,
+  @Override public MeasureRawColumnChunk readRawMeasureChunk(FileReader fileReader,
       int blockletColumnIndex) throws IOException {
     int dataLength = 0;
     // to calculate the length of the data to be read
@@ -110,7 +110,7 @@ public class CompressedMsrChunkFileBasedPageLevelReaderV3
    * @param endColumnBlockletIndex   blocklet index of the last measure column
    * @return MeasureRawColumnChunk array
    */
-  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileHolder fileReader,
+  protected MeasureRawColumnChunk[] readRawMeasureChunksInGroup(FileReader fileReader,
       int startColumnBlockletIndex, int endColumnBlockletIndex) throws IOException {
     // create raw chunk for each measure column
     MeasureRawColumnChunk[] measureDataChunk =
@@ -130,7 +130,7 @@ public class CompressedMsrChunkFileBasedPageLevelReaderV3
    * @param pageNumber            number
    * @return DimensionColumnDataChunk
    */
-  @Override public ColumnPage convertToColumnPage(
+  @Override public ColumnPage decodeColumnPage(
       MeasureRawColumnChunk rawColumnPage, int pageNumber)
       throws IOException, MemoryException {
     // data chunk of blocklet column

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
index fbdb499..c89ecc3 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/ColumnPageWrapper.java
@@ -17,48 +17,45 @@
 
 package org.apache.carbondata.core.datastore.chunk.store;
 
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
 
-public class ColumnPageWrapper implements DimensionColumnDataChunk {
+public class ColumnPageWrapper implements DimensionColumnPage {
 
   private ColumnPage columnPage;
-  private int columnValueSize;
 
-  public ColumnPageWrapper(ColumnPage columnPage, int columnValueSize) {
+  public ColumnPageWrapper(ColumnPage columnPage) {
     this.columnPage = columnPage;
-    this.columnValueSize = columnValueSize;
   }
 
   @Override
-  public int fillChunkData(byte[] data, int offset, int columnIndex,
-      KeyStructureInfo restructuringInfo) {
+  public int fillRawData(int rowId, int offset, byte[] data, KeyStructureInfo restructuringInfo) {
     throw new UnsupportedOperationException("internal error");
   }
 
   @Override
-  public int fillConvertedChunkData(int rowId, int columnIndex, int[] row,
+  public int fillSurrogateKey(int rowId, int chunkIndex, int[] outputSurrogateKey,
       KeyStructureInfo restructuringInfo) {
     throw new UnsupportedOperationException("internal error");
   }
 
   @Override
-  public int fillConvertedChunkData(ColumnVectorInfo[] vectorInfo, int column,
+  public int fillVector(ColumnVectorInfo[] vectorInfo, int chunkIndex,
       KeyStructureInfo restructuringInfo) {
     throw new UnsupportedOperationException("internal error");
   }
 
   @Override
-  public int fillConvertedChunkData(int[] rowMapping, ColumnVectorInfo[] vectorInfo, int column,
+  public int fillVector(int[] filteredRowId, ColumnVectorInfo[] vectorInfo, int chunkIndex,
       KeyStructureInfo restructuringInfo) {
     throw new UnsupportedOperationException("internal error");
   }
 
   @Override
-  public byte[] getChunkData(int columnIndex) {
-    return columnPage.getBytes(columnIndex);
+  public byte[] getChunkData(int rowId) {
+    return columnPage.getBytes(rowId);
   }
 
   @Override
@@ -66,7 +63,7 @@ public class ColumnPageWrapper implements DimensionColumnDataChunk {
     throw new UnsupportedOperationException("internal error");
   }
 
-  @Override public int getInvertedReverseIndex(int invertedIndex) {
+  @Override public int getInvertedReverseIndex(int rowId) {
     throw new UnsupportedOperationException("internal error");
   }
 
@@ -76,17 +73,12 @@ public class ColumnPageWrapper implements DimensionColumnDataChunk {
   }
 
   @Override
-  public int getColumnValueSize() {
-    return columnValueSize;
-  }
-
-  @Override
   public boolean isExplicitSorted() {
     return false;
   }
 
   @Override
-  public int compareTo(int index, byte[] compareValue) {
+  public int compareTo(int rowId, byte[] compareValue) {
     throw new UnsupportedOperationException("internal error");
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
index 5072c75..28aed5b 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/DimensionDataChunkStore.java
@@ -69,10 +69,10 @@ public interface DimensionDataChunkStore {
 
   /**
    * Below method will be used to get the reverse Inverted Index
-   * @param invertedIndex
+   * @param rowId
    * @return reverse Inverted Index
    */
-  int getInvertedReverseIndex(int invertedIndex);
+  int getInvertedReverseIndex(int rowId);
 
   /**
    * Below method will be used to get the surrogate key of the
@@ -102,9 +102,9 @@ public interface DimensionDataChunkStore {
   /**
    * to compare the two byte array
    *
-   * @param index        index of first byte array
+   * @param rowId        index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
-  int compareTo(int index, byte[] compareValue);
+  int compareTo(int rowId, byte[] compareValue);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java
index 8656878..41218d0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeFixedLengthDimensionDataChunkStore.java
@@ -97,13 +97,13 @@ public class SafeFixedLengthDimensionDataChunkStore extends SafeAbsractDimension
   /**
    * to compare the two byte array
    *
-   * @param index        index of first byte array
+   * @param rowId        index of first byte array
    * @param compareValue value of to be compared
    * @return compare result
    */
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
     return ByteUtil.UnsafeComparer.INSTANCE
-        .compareTo(data, index * columnValueSize, columnValueSize, compareValue, 0,
+        .compareTo(data, rowId * columnValueSize, columnValueSize, compareValue, 0,
             columnValueSize);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
index db83198..f498c6e 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/chunk/store/impl/safe/SafeVariableLengthDimensionDataChunkStore.java
@@ -162,7 +162,7 @@ public class SafeVariableLengthDimensionDataChunkStore extends SafeAbsractDimens
     }
   }
 
-  @Override public int compareTo(int index, byte[] compareValue) {
+  @Override public int compareTo(int rowId, byte[] compareValue) {
     // now to get the row from memory block we need to do following thing
     // 1. first get the current offset
     // 2. if it's not a last row- get the next row offset
@@ -171,11 +171,11 @@ public class SafeVariableLengthDimensionDataChunkStore extends SafeAbsractDimens
     // length
 
     // get the offset of set of data
-    int currentDataOffset = dataOffsets[index];
+    int currentDataOffset = dataOffsets[rowId];
     short length = 0;
     // calculating the length of data
-    if (index < numberOfRows - 1) {
-      length = (short) (dataOffsets[index + 1] - (currentDataOffset
+    if (rowId < numberOfRows - 1) {
+      length = (short) (dataOffsets[rowId + 1] - (currentDataOffset
           + CarbonCommonConstants.SHORT_SIZE_IN_BYTE));
     } else {
       // for last record


[05/49] carbondata git commit: [CARBONDATA-1968] Add external table support

Posted by ja...@apache.org.
[CARBONDATA-1968] Add external table support

This PR adds support for creating external table with existing carbondata files, using Hive syntax.
CREATE EXTERNAL TABLE tableName STORED BY 'carbondata' LOCATION 'path'

This closes #1749


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

Branch: refs/heads/carbonstore-rebase4
Commit: c2095089c5a6e203f44158248178fa11ce627925
Parents: bf3602f
Author: Jacky Li <ja...@qq.com>
Authored: Tue Jan 2 23:46:14 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Mon Feb 26 23:55:29 2018 +0800

----------------------------------------------------------------------
 .../core/metadata/schema/table/CarbonTable.java |  9 ++
 .../createTable/TestCreateExternalTable.scala   | 91 ++++++++++++++++++++
 .../TestDataWithDicExcludeAndInclude.scala      | 10 ---
 .../command/table/CarbonDropTableCommand.scala  |  5 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala | 64 +++++++++-----
 5 files changed, 147 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c2095089/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 09ff440..6036569 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -785,6 +785,15 @@ public class CarbonTable implements Serializable {
         && !tableInfo.getParentRelationIdentifiers().isEmpty();
   }
 
+  /**
+   * Return true if this is an external table (table with property "_external"="true", this is
+   * an internal table property set during table creation)
+   */
+  public boolean isExternalTable() {
+    String external = tableInfo.getFactTable().getTableProperties().get("_external");
+    return external != null && external.equalsIgnoreCase("true");
+  }
+
   public long size() throws IOException {
     Map<String, Long> dataIndexSize = CarbonUtil.calculateDataIndexSize(this);
     Long dataSize = dataIndexSize.get(CarbonCommonConstants.CARBON_TOTAL_DATA_SIZE);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c2095089/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
new file mode 100644
index 0000000..67370eb
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.createTable
+
+import java.io.File
+
+import org.apache.spark.sql.{AnalysisException, CarbonEnv}
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestCreateExternalTable extends QueryTest with BeforeAndAfterAll {
+
+  var originDataPath: String = _
+
+  override def beforeAll(): Unit = {
+    sql("DROP TABLE IF EXISTS origin")
+    // create carbon table and insert data
+    sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+    sql("INSERT INTO origin select 100,'spark'")
+    sql("INSERT INTO origin select 200,'hive'")
+    originDataPath = s"$storeLocation/origin"
+  }
+
+  override def afterAll(): Unit = {
+    sql("DROP TABLE IF EXISTS origin")
+  }
+
+  test("create external table with existing files") {
+    assert(new File(originDataPath).exists())
+    sql("DROP TABLE IF EXISTS source")
+
+    // create external table with existing files
+    sql(
+      s"""
+         |CREATE EXTERNAL TABLE source
+         |STORED BY 'carbondata'
+         |LOCATION '$storeLocation/origin'
+       """.stripMargin)
+    checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
+
+    val carbonTable = CarbonEnv.getCarbonTable(None, "source")(sqlContext.sparkSession)
+    assert(carbonTable.isExternalTable)
+    
+    sql("DROP TABLE IF EXISTS source")
+
+    // DROP TABLE should not delete data
+    assert(new File(originDataPath).exists())
+  }
+
+  test("create external table with empty folder") {
+    val exception = intercept[AnalysisException] {
+      sql(
+        s"""
+           |CREATE EXTERNAL TABLE source
+           |STORED BY 'carbondata'
+           |LOCATION './nothing'
+         """.stripMargin)
+    }
+    assert(exception.getMessage().contains("Invalid table path provided"))
+  }
+
+  test("create external table with CTAS") {
+    val exception = intercept[AnalysisException] {
+      sql(
+        """
+          |CREATE EXTERNAL TABLE source
+          |STORED BY 'carbondata'
+          |LOCATION './nothing'
+          |AS
+          | SELECT * FROM origin
+        """.stripMargin)
+    }
+    assert(exception.getMessage().contains("Create external table as select"))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c2095089/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala
index c788857..201da39 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataWithDicExcludeAndInclude.scala
@@ -90,16 +90,6 @@ class TestLoadDataWithDictionaryExcludeAndInclude extends QueryTest with BeforeA
     )
   }
 
-  test("test create external table should fail") {
-    assert(intercept[AnalysisException](
-      sql(
-        """
-          | CREATE EXTERNAL TABLE t1 (id string, value int)
-          | STORED BY 'carbondata'
-        """.stripMargin)
-    ).message.contains("Operation not allowed: CREATE EXTERNAL TABLE"))
-  }
-
   override def afterAll {
     dropTable
     CarbonProperties.getInstance()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c2095089/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
index 7c895ab..8001a93 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
@@ -147,7 +147,10 @@ case class CarbonDropTableCommand(
       // delete the table folder
       val tablePath = carbonTable.getTablePath
       val fileType = FileFactory.getFileType(tablePath)
-      if (FileFactory.isFileExist(tablePath, fileType)) {
+
+      // delete table data only if it is not external table
+      if (FileFactory.isFileExist(tablePath, fileType) &&
+          !carbonTable.isExternalTable) {
         val file = FileFactory.getCarbonFile(tablePath, fileType)
         CarbonUtil.deleteFoldersAndFilesSilent(file)
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c2095089/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
index 4b77417..ad6d0c7 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
@@ -32,6 +32,8 @@ import org.apache.spark.sql.types.StructField
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.CarbonReflectionUtils
 
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.hadoop.util.SchemaReader
 import org.apache.carbondata.spark.CarbonOption
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.CommonUtil
@@ -164,9 +166,6 @@ class CarbonHelperSqlAstBuilder(conf: SQLConf,
     if (bucketSpecContext != null) {
       operationNotAllowed("CREATE TABLE ... CLUSTERED BY", bucketSpecContext)
     }
-    if (external) {
-      operationNotAllowed("CREATE EXTERNAL TABLE", tableHeader)
-    }
 
     val cols = Option(columns).toSeq.flatMap(visitColTypeList)
     val properties = getPropertyKeyValues(tablePropertyList)
@@ -231,6 +230,10 @@ class CarbonHelperSqlAstBuilder(conf: SQLConf,
           operationNotAllowed(
             "Schema may not be specified in a Create Table As Select (CTAS) statement", columns)
         }
+        // external table is not allow
+        if (external) {
+          operationNotAllowed("Create external table as select", tableHeader)
+        }
         fields = parser
           .getFields(CarbonEnv.getInstance(sparkSession).carbonMetastore
             .getSchemaFromUnresolvedRelation(sparkSession, Some(q).get))
@@ -242,29 +245,48 @@ class CarbonHelperSqlAstBuilder(conf: SQLConf,
     }
     // validate tblProperties
     val bucketFields = parser.getBucketFields(tableProperties, fields, options)
-    // prepare table model of the collected tokens
-    val tableModel: TableModel = parser.prepareTableModel(
-      ifNotExists,
-      convertDbNameToLowerCase(tableIdentifier.database),
-      tableIdentifier.table.toLowerCase,
-      fields,
-      partitionFields,
-      tableProperties,
-      bucketFields,
-      isAlterFlow = false,
-      tableComment)
 
+    val tableInfo = if (external) {
+      // read table info from schema file in the provided table path
+      val identifier = AbsoluteTableIdentifier.from(
+        tablePath.get,
+        CarbonEnv.getDatabaseName(tableIdentifier.database)(sparkSession),
+        tableIdentifier.table)
+      val table = try {
+        SchemaReader.getTableInfo(identifier)
+      } catch {
+        case e: Throwable =>
+          operationNotAllowed(s"Invalid table path provided: ${tablePath.get} ", tableHeader)
+      }
+      // set "_external" property, so that DROP TABLE will not delete the data
+      table.getFactTable.getTableProperties.put("_external", "true")
+      table
+    } else {
+      // prepare table model of the collected tokens
+      val tableModel: TableModel = parser.prepareTableModel(
+        ifNotExists,
+        convertDbNameToLowerCase(tableIdentifier.database),
+        tableIdentifier.table.toLowerCase,
+        fields,
+        partitionFields,
+        tableProperties,
+        bucketFields,
+        isAlterFlow = false,
+        tableComment)
+      TableNewProcessor(tableModel)
+    }
     selectQuery match {
       case query@Some(q) =>
         CarbonCreateTableAsSelectCommand(
-          TableNewProcessor(tableModel),
-          query.get,
-          tableModel.ifNotExistsSet,
-          tablePath)
+          tableInfo = tableInfo,
+          query = query.get,
+          ifNotExistsSet = ifNotExists,
+          tableLocation = tablePath)
       case _ =>
-        CarbonCreateTableCommand(TableNewProcessor(tableModel),
-          tableModel.ifNotExistsSet,
-          tablePath)
+        CarbonCreateTableCommand(
+          tableInfo = tableInfo,
+          ifNotExistsSet = ifNotExists,
+          tableLocation = tablePath)
     }
   }
 


[48/49] carbondata git commit: [CARBONDATA-1114][Tests] Fix bugs in tests in windows env

Posted by ja...@apache.org.
[CARBONDATA-1114][Tests] Fix bugs in tests in windows env

Fix bugs in tests that will cause failure under windows env

This closes #1994


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

Branch: refs/heads/carbonstore-rebase4
Commit: de36a5d85d992dee4380ae228e6252baf1c31fe9
Parents: fa1a100
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Sat Feb 24 21:18:17 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:24:02 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/locks/LocalFileLock.java    | 30 +++++++++++---------
 .../store/impl/DFSFileReaderImplUnitTest.java   | 11 ++++---
 .../store/impl/FileFactoryImplUnitTest.java     | 28 +++++++++++++-----
 .../filesystem/HDFSCarbonFileTest.java          |  3 +-
 .../filesystem/LocalCarbonFileTest.java         | 20 +++++++++----
 .../BooleanDataTypesInsertTest.scala            |  5 ++--
 .../carbondata/lcm/locks/LocalFileLockTest.java |  2 +-
 .../loading/csvinput/CSVInputFormatTest.java    |  1 +
 8 files changed, 64 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/de36a5d8/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
index 75ea074..cb80877 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/LocalFileLock.java
@@ -17,17 +17,20 @@
 
 package org.apache.carbondata.core.locks;
 
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.channels.FileChannel;
 import java.nio.channels.FileLock;
 import java.nio.channels.OverlappingFileLockException;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * This class handles the file locking in the local file system.
@@ -40,11 +43,6 @@ public class LocalFileLock extends AbstractCarbonLock {
   private String location;
 
   /**
-   * fileOutputStream of the local lock file
-   */
-  private FileOutputStream fileOutputStream;
-
-  /**
    * channel is the FileChannel of the lock file.
    */
   private FileChannel channel;
@@ -104,8 +102,8 @@ public class LocalFileLock extends AbstractCarbonLock {
         FileFactory.createNewLockFile(lockFilePath, FileFactory.getFileType(location));
       }
 
-      fileOutputStream = new FileOutputStream(lockFilePath);
-      channel = fileOutputStream.getChannel();
+      channel = FileChannel.open(Paths.get(lockFilePath), StandardOpenOption.WRITE,
+          StandardOpenOption.APPEND);
       try {
         fileLock = channel.tryLock();
       } catch (OverlappingFileLockException e) {
@@ -137,11 +135,17 @@ public class LocalFileLock extends AbstractCarbonLock {
     } catch (IOException e) {
       status = false;
     } finally {
-      if (null != fileOutputStream) {
-        try {
-          fileOutputStream.close();
-        } catch (IOException e) {
-          LOGGER.error(e.getMessage());
+      CarbonUtil.closeStreams(channel);
+
+      // deleting the lock file after releasing the lock.
+      if (null != lockFilePath) {
+        CarbonFile lockFile = FileFactory.getCarbonFile(lockFilePath,
+            FileFactory.getFileType(lockFilePath));
+        if (!lockFile.exists() || lockFile.delete()) {
+          LOGGER.info("Successfully deleted the lock file " + lockFilePath);
+        } else {
+          LOGGER.error("Not able to delete the lock file " + lockFilePath);
+          status = false;
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de36a5d8/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
index da61a94..30144c1 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
@@ -38,7 +38,6 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 
 public class DFSFileReaderImplUnitTest {
-
   private static DFSFileReaderImpl dfsFileHolder;
   private static String fileName;
   private static String fileNameWithEmptyContent;
@@ -50,10 +49,8 @@ public class DFSFileReaderImplUnitTest {
     file = new File("Test.carbondata");
     fileWithEmptyContent = new File("TestEXception.carbondata");
 
-    if (!file.exists()) try {
-      file.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
+    if (file.exists()) {
+      file.delete();
     }
     if (!fileWithEmptyContent.exists()) try {
       fileWithEmptyContent.createNewFile();
@@ -61,10 +58,12 @@ public class DFSFileReaderImplUnitTest {
       e.printStackTrace();
     }
     try {
-      FileOutputStream of = new FileOutputStream(file, true);
+      FileOutputStream of = new FileOutputStream(file, false);
       BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
       br.write("Hello World");
       br.close();
+      of.flush();
+      of.close();
     } catch (Exception e) {
       e.getMessage();
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de36a5d8/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java
index 65590d6..0e7d1c9 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileFactoryImplUnitTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
 
+import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
@@ -43,6 +44,10 @@ public class FileFactoryImplUnitTest {
 
   @AfterClass
   public static void tearDown() {
+    cleanUp();
+  }
+
+  private static void cleanUp() {
     File file = new File(filePath);
     if (file.exists()) {
       file.delete();
@@ -83,17 +88,17 @@ public class FileFactoryImplUnitTest {
   }
 
   @Test public void testCreateNewFileWithDefaultFileType() throws IOException {
-    tearDown();
+    cleanUp();
     assertTrue(FileFactory.createNewFile(filePath, FileFactory.FileType.LOCAL));
   }
 
   @Test public void testCreateNewLockFileWithDefaultFileType() throws IOException {
-    tearDown();
+    cleanUp();
     assertTrue(FileFactory.createNewLockFile(filePath, FileFactory.FileType.LOCAL));
   }
 
   @Test public void testCreateNewLockFileWithViewFsFileType() throws IOException {
-    tearDown();
+    cleanUp();
     assertTrue(FileFactory.createNewLockFile(filePath, FileFactory.FileType.VIEWFS));
   }
 
@@ -129,20 +134,29 @@ public class FileFactoryImplUnitTest {
     assertTrue(FileFactory.mkdirs(filePath, FileFactory.FileType.VIEWFS));
   }
 
-  @Test public void testGetDataOutputStreamUsingAppendeForException() {
+  @Test public void testGetDataOutputStreamUsingAppendeForException() throws IOException {
+    DataOutputStream outputStream = null;
     try {
-      FileFactory.getDataOutputStreamUsingAppend(filePath, FileFactory.FileType.VIEWFS);
+      outputStream = FileFactory.getDataOutputStreamUsingAppend(filePath, FileFactory.FileType.VIEWFS);
     } catch (Exception exception) {
       assertEquals("Not supported", exception.getMessage());
+    } finally {
+      if (null != outputStream) {
+        outputStream.close();
+      }
     }
   }
 
   @Test public void getDataOutputStreamForVIEWFSType() throws IOException {
-    assertNotNull(FileFactory.getDataOutputStream(filePath, FileFactory.FileType.VIEWFS));
+    DataOutputStream outputStream = FileFactory.getDataOutputStream(filePath, FileFactory.FileType.VIEWFS);
+    assertNotNull(outputStream);
+    outputStream.close();
   }
 
   @Test public void getDataOutputStreamForLocalType() throws IOException {
-    assertNotNull(FileFactory.getDataOutputStream(filePath, FileFactory.FileType.LOCAL));
+    DataOutputStream outputStream = FileFactory.getDataOutputStream(filePath, FileFactory.FileType.LOCAL);
+    assertNotNull(outputStream);
+    outputStream.close();
   }
 
   @Test public void testGetCarbonFile() throws IOException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de36a5d8/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java
index 4018123..42d4afa 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java
@@ -58,7 +58,7 @@ public class HDFSCarbonFileTest {
     @BeforeClass
     static public void setUp() throws IOException {
         Configuration config = new Configuration();
-//adding local hadoop configuration
+        // adding local hadoop configuration
         config.addResource(new Path("core-site.xml"));
         config.addResource(new Path("hdfs-site.xml"));
         fileName = "Test.carbondata"; //this path is HDFS path
@@ -75,6 +75,7 @@ public class HDFSCarbonFileTest {
             BufferedWriter br = new BufferedWriter(new OutputStreamWriter(os, "UTF-8"));
             br.write("Hello World");
             br.close();
+            os.close();
             fs.close();
 
             fileStatus = new FileStatus(12L, true, 60, 120l, 180L, new Path(fileName));

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de36a5d8/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java
index 96ef106..14f9fe2 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/LocalCarbonFileTest.java
@@ -29,6 +29,7 @@ import sun.nio.ch.FileChannelImpl;
 import java.io.*;
 import java.nio.channels.ReadableByteChannel;
 import java.util.Objects;
+import java.util.UUID;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -44,8 +45,8 @@ public class LocalCarbonFileTest {
 
     @BeforeClass
     static public void setUp() {
-        file = new File("Test.carbondata");
-        dir = new File("Testdir.carbondata");
+        file = new File("TestLocalCarbonFile");
+        dir = new File("TestLocalCarbonDir");
         if (!file.exists())
             try {
                 file.createNewFile();
@@ -60,6 +61,7 @@ public class LocalCarbonFileTest {
             byte[] bytes = "core java api".getBytes();
 
             oFile.write(bytes);
+            oFile.close();
         } catch (FileNotFoundException e) {
             e.printStackTrace();
             localCarbonFile = new LocalCarbonFile(file);
@@ -121,8 +123,9 @@ public class LocalCarbonFileTest {
     @Test
     public void testRenameForce() {
         localCarbonFile = new LocalCarbonFile(file);
-        assertTrue(localCarbonFile.renameForce("Testdb.carbon"));
-        File file1 = new File("Testdb.carbon");
+        String destFile = "TestRename" + UUID.randomUUID().toString();
+        assertTrue(localCarbonFile.renameForce(destFile));
+        File file1 = new File(destFile);
         if (file1.exists()) {
             file1.delete();
         }
@@ -131,7 +134,12 @@ public class LocalCarbonFileTest {
     @Test
     public void testRenameTo() {
         localCarbonFile = new LocalCarbonFile(file);
-        assertTrue(!localCarbonFile.renameTo("Testdb.carbon"));
+        String destFile = "TestRename" + UUID.randomUUID().toString();
+        assertTrue(!localCarbonFile.renameTo(destFile));
+        File file1 = new File(destFile);
+        if (file1.exists()) {
+            file1.delete();
+        }
     }
 
     @Test
@@ -463,6 +471,6 @@ public class LocalCarbonFileTest {
 
         localCarbonFile = new LocalCarbonFile("demo.txt");
 
-        assertEquals(localCarbonFile.renameForce("Test.carbondata"), true);
+        assertEquals(localCarbonFile.renameForce("renameToFile"), true);
     }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de36a5d8/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala
index f8cfa6b..45edd3d 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/booleantype/BooleanDataTypesInsertTest.scala
@@ -23,6 +23,7 @@ import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.CarbonProperties
 
 class BooleanDataTypesInsertTest extends QueryTest with BeforeAndAfterEach with BeforeAndAfterAll {
@@ -555,7 +556,7 @@ class BooleanDataTypesInsertTest extends QueryTest with BeforeAndAfterEach with
 
     sql(
       s"""
-         | LOAD DATA LOCAL INPATH '${storeLocation}'
+         | LOAD DATA LOCAL INPATH '${FileFactory.getUpdatedFilePath(storeLocation)}'
          | INTO TABLE hive_table
            """.stripMargin)
 
@@ -923,7 +924,7 @@ class BooleanDataTypesInsertTest extends QueryTest with BeforeAndAfterEach with
 
     sql(
       s"""
-         | LOAD DATA LOCAL INPATH '${storeLocation}'
+         | LOAD DATA LOCAL INPATH '${FileFactory.getUpdatedFilePath(storeLocation)}'
          | INTO TABLE hive_table
            """.stripMargin)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de36a5d8/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java b/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
index 812db5c..8d5f3d4 100644
--- a/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
+++ b/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
@@ -65,7 +65,7 @@ public class LocalFileLockTest {
 
     Assert.assertTrue(localLock1.unlock());
     Assert.assertTrue(localLock2.lock());
-
+    Assert.assertTrue(localLock2.unlock());
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/de36a5d8/processing/src/test/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormatTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormatTest.java b/processing/src/test/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormatTest.java
index 14c680e..9a9c662 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormatTest.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormatTest.java
@@ -147,6 +147,7 @@ public class CSVInputFormatTest extends TestCase {
     FileOutputFormat.setOutputPath(job, new Path(output.getCanonicalPath()));
 
     Assert.assertTrue(job.waitForCompletion(true));
+    deleteOutput(output);
   }
 
   private void prepareConf(Configuration conf) {


[15/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
index 0cb2918..099fffd 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/blocklet/SegmentInfo.java
@@ -29,31 +29,12 @@ public class SegmentInfo implements Serializable {
   private static final long serialVersionUID = -1749874611112709431L;
 
   /**
-   * number of column in the segment
-   */
-  private int numberOfColumns;
-
-  /**
    * cardinality of each columns
    * column which is not participating in the multidimensional key cardinality will be -1;
    */
   private int[] columnCardinality;
 
   /**
-   * @return the numberOfColumns
-   */
-  public int getNumberOfColumns() {
-    return numberOfColumns;
-  }
-
-  /**
-   * @param numberOfColumns the numberOfColumns to set
-   */
-  public void setNumberOfColumns(int numberOfColumns) {
-    this.numberOfColumns = numberOfColumns;
-  }
-
-  /**
    * @return the columnCardinality
    */
   public int[] getColumnCardinality() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 6036569..d17d865 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -19,7 +19,13 @@ package org.apache.carbondata.core.metadata.schema.table;
 
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -33,7 +39,10 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonImplicitDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.scan.model.QueryModel;
+import org.apache.carbondata.core.scan.model.QueryProjection;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DataTypeConverter;
 import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
@@ -136,10 +145,7 @@ public class CarbonTable implements Serializable {
   /**
    * During creation of TableInfo from hivemetastore the DataMapSchemas and the columns
    * DataTypes are not converted to the appropriate child classes.
-   *
    * This method will cast the same to the appropriate classes
-   *
-   * @param tableInfo
    */
   public static void updateTableInfo(TableInfo tableInfo) {
     List<DataMapSchema> dataMapSchemas = new ArrayList<>();
@@ -153,8 +159,9 @@ public class CarbonTable implements Serializable {
     }
     tableInfo.setDataMapSchemaList(dataMapSchemas);
     for (ColumnSchema columnSchema : tableInfo.getFactTable().getListOfColumns()) {
-      columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(),
-          columnSchema.getPrecision(), columnSchema.getScale()));
+      columnSchema.setDataType(
+          DataTypeUtil.valueOf(
+              columnSchema.getDataType(), columnSchema.getPrecision(), columnSchema.getScale()));
     }
     List<DataMapSchema> childSchema = tableInfo.getDataMapSchemaList();
     for (DataMapSchema dataMapSchema : childSchema) {
@@ -168,10 +175,11 @@ public class CarbonTable implements Serializable {
       }
     }
     if (tableInfo.getFactTable().getBucketingInfo() != null) {
-      for (ColumnSchema columnSchema : tableInfo.getFactTable()
-          .getBucketingInfo().getListOfColumns()) {
-        columnSchema.setDataType(DataTypeUtil.valueOf(columnSchema.getDataType(),
-            columnSchema.getPrecision(), columnSchema.getScale()));
+      for (ColumnSchema columnSchema :
+          tableInfo.getFactTable().getBucketingInfo().getListOfColumns()) {
+        columnSchema.setDataType(
+            DataTypeUtil.valueOf(
+                columnSchema.getDataType(), columnSchema.getPrecision(), columnSchema.getScale()));
       }
     }
     if (tableInfo.getFactTable().getPartitionInfo() != null) {
@@ -211,6 +219,7 @@ public class CarbonTable implements Serializable {
 
   /**
    * fill columns as per user provided order
+   *
    * @param tableName
    */
   private void fillCreateOrderColumn(String tableName) {
@@ -229,7 +238,6 @@ public class CarbonTable implements Serializable {
     this.createOrderColumn.put(tableName, columns);
   }
 
-
   /**
    * Fill allDimensions and allMeasures for carbon table
    *
@@ -254,8 +262,8 @@ public class CarbonTable implements Serializable {
       if (columnSchema.isDimensionColumn()) {
         if (columnSchema.getNumberOfChild() > 0) {
           CarbonDimension complexDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++,
-                  columnSchema.getSchemaOrdinal(), -1, -1, ++complexTypeOrdinal);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
+                  -1, -1, ++complexTypeOrdinal);
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           allDimensions.add(complexDimension);
           dimensionOrdinal =
@@ -268,9 +276,8 @@ public class CarbonTable implements Serializable {
             this.numberOfSortColumns++;
           }
           if (!columnSchema.getEncodingList().contains(Encoding.DICTIONARY)) {
-            CarbonDimension dimension =
-                new CarbonDimension(columnSchema, dimensionOrdinal++,
-                    columnSchema.getSchemaOrdinal(), -1, -1, -1);
+            CarbonDimension dimension = new CarbonDimension(
+                columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), -1, -1, -1);
             if (!columnSchema.isInvisible() && columnSchema.isSortColumn()) {
               this.numberOfNoDictSortColumns++;
             }
@@ -278,25 +285,25 @@ public class CarbonTable implements Serializable {
             primitiveDimensions.add(dimension);
           } else if (columnSchema.getEncodingList().contains(Encoding.DICTIONARY)
               && columnSchema.getColumnGroupId() == -1) {
-            CarbonDimension dimension =
-                new CarbonDimension(columnSchema, dimensionOrdinal++,
-                    columnSchema.getSchemaOrdinal(), keyOrdinal++, -1, -1);
+            CarbonDimension dimension = new CarbonDimension(
+                columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), keyOrdinal++,
+                -1, -1);
             allDimensions.add(dimension);
             primitiveDimensions.add(dimension);
           } else {
             columnGroupOrdinal =
                 previousColumnGroupId == columnSchema.getColumnGroupId() ? ++columnGroupOrdinal : 0;
             previousColumnGroupId = columnSchema.getColumnGroupId();
-            CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++,
-                columnSchema.getSchemaOrdinal(), keyOrdinal++,
+            CarbonDimension dimension = new CarbonDimension(
+                columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(), keyOrdinal++,
                 columnGroupOrdinal, -1);
             allDimensions.add(dimension);
             primitiveDimensions.add(dimension);
           }
         }
       } else {
-        allMeasures.add(new CarbonMeasure(columnSchema, measureOrdinal++,
-            columnSchema.getSchemaOrdinal()));
+        allMeasures.add(
+            new CarbonMeasure(columnSchema, measureOrdinal++, columnSchema.getSchemaOrdinal()));
       }
     }
     fillVisibleDimensions(tableSchema.getTableName());
@@ -347,8 +354,8 @@ public class CarbonTable implements Serializable {
       if (columnSchema.isDimensionColumn()) {
         if (columnSchema.getNumberOfChild() > 0) {
           CarbonDimension complexDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++,
-                  columnSchema.getSchemaOrdinal(), -1, -1, -1);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
+                  -1, -1, -1);
           complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
           parentDimension.getListOfChildDimensions().add(complexDimension);
           dimensionOrdinal =
@@ -356,8 +363,8 @@ public class CarbonTable implements Serializable {
                   listOfColumns, complexDimension, primitiveDimensions);
         } else {
           CarbonDimension carbonDimension =
-              new CarbonDimension(columnSchema, dimensionOrdinal++,
-                  columnSchema.getSchemaOrdinal(), -1, -1, -1);
+              new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
+                  -1, -1, -1);
           parentDimension.getListOfChildDimensions().add(carbonDimension);
           primitiveDimensions.add(carbonDimension);
         }
@@ -569,6 +576,7 @@ public class CarbonTable implements Serializable {
     }
     return null;
   }
+
   /**
    * gets all children dimension for complex type
    *
@@ -771,18 +779,13 @@ public class CarbonTable implements Serializable {
     return dimensionOrdinalMax;
   }
 
-  public void setDimensionOrdinalMax(int dimensionOrdinalMax) {
-    this.dimensionOrdinalMax = dimensionOrdinalMax;
-  }
-
-
   public boolean hasDataMapSchema() {
     return hasDataMapSchema;
   }
 
   public boolean isChildDataMap() {
-    return null != tableInfo.getParentRelationIdentifiers()
-        && !tableInfo.getParentRelationIdentifiers().isEmpty();
+    return null != tableInfo.getParentRelationIdentifiers() &&
+        !tableInfo.getParentRelationIdentifiers().isEmpty();
   }
 
   /**
@@ -806,4 +809,63 @@ public class CarbonTable implements Serializable {
     }
     return dataSize + indexSize;
   }
+
+  /**
+   * Create a new QueryModel with projection all columns in the table.
+   */
+  public QueryModel createQueryModelWithProjectAllColumns(DataTypeConverter converter) {
+    QueryProjection projection = new QueryProjection();
+
+    List<CarbonDimension> dimensions = getDimensionByTableName(getTableName());
+    for (int i = 0; i < dimensions.size(); i++) {
+      projection.addDimension(dimensions.get(i), i);
+    }
+    List<CarbonMeasure> measures = getMeasureByTableName(getTableName());
+    for (int i = 0; i < measures.size(); i++) {
+      projection.addMeasure(measures.get(i), i);
+    }
+    QueryModel model = QueryModel.newInstance(this);
+    model.setProjection(projection);
+    model.setConverter(converter);
+    return model;
+  }
+
+  /**
+   * Create a new QueryModel with specified projection
+   */
+  public QueryModel createQueryWithProjection(String[] projectionColumnNames,
+      DataTypeConverter converter) {
+    QueryProjection projection = createProjection(projectionColumnNames);
+    QueryModel queryModel = QueryModel.newInstance(this);
+    queryModel.setProjection(projection);
+    queryModel.setConverter(converter);
+    return queryModel;
+  }
+
+  private QueryProjection createProjection(String[] projectionColumnNames) {
+    String factTableName = getTableName();
+    QueryProjection projection = new QueryProjection();
+    // fill dimensions
+    // If columns are null, set all dimensions and measures
+    int i = 0;
+    if (projectionColumnNames != null) {
+      for (String projectionColumnName : projectionColumnNames) {
+        CarbonDimension dimension = getDimensionByName(factTableName, projectionColumnName);
+        if (dimension != null) {
+          projection.addDimension(dimension, i);
+          i++;
+        } else {
+          CarbonMeasure measure = getMeasureByName(factTableName, projectionColumnName);
+          if (measure == null) {
+            throw new RuntimeException(projectionColumnName +
+                " column not found in the table " + factTableName);
+          }
+          projection.addMeasure(measure, i);
+          i++;
+        }
+      }
+    }
+
+    return projection;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
index 20e539b..9a1dad1 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/RelationIdentifier.java
@@ -26,18 +26,6 @@ import java.io.Serializable;
  */
 public class RelationIdentifier implements Serializable, Writable {
 
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  public void setTableId(String tableId) {
-    this.tableId = tableId;
-  }
-
   private String databaseName;
 
   private String tableName;
@@ -50,10 +38,6 @@ public class RelationIdentifier implements Serializable, Writable {
     this.tableId = tableId;
   }
 
-  public RelationIdentifier() {
-    this(null, null, null);
-  }
-
   public String getDatabaseName() {
     return databaseName;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
index 4deafd4..0d796c7 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java
@@ -24,7 +24,11 @@ import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.io.Serializable;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
index 13e7d49..c888418 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonColumn.java
@@ -45,7 +45,7 @@ public class CarbonColumn implements Serializable {
   /**
    * order in which user has created table
    */
-  protected int schemaOrdinal;
+  private int schemaOrdinal;
 
   /**
    * Column identifier

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
index b4c052f..d15cde7 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/CarbonDimension.java
@@ -122,18 +122,6 @@ public class CarbonDimension extends CarbonColumn {
     return getEncoder().contains(Encoding.DICTIONARY);
   }
 
-  public int getNumDimensionsExpanded() {
-    if (listOfChildDimensions == null) {
-      // there is no child, return 1 column
-      return 1;
-    }
-    int columnCount = 1;
-    for (CarbonDimension dimension: listOfChildDimensions) {
-      columnCount += dimension.getNumDimensionsExpanded();
-    }
-    return columnCount;
-  }
-
   /**
    * @return is column participated in sorting or not
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
index 18eae11..7289c98 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
@@ -128,8 +128,6 @@ public class CarbonUpdateUtil {
       lockStatus = updateLock.lockWithRetries();
       if (lockStatus) {
 
-        AbsoluteTableIdentifier absoluteTableIdentifier = table.getAbsoluteTableIdentifier();
-
         // read the existing file if present and update the same.
         SegmentUpdateDetails[] oldDetails = segmentUpdateStatusManager
                 .getUpdateStatusDetails();
@@ -389,16 +387,6 @@ public class CarbonUpdateUtil {
         .substring(0, completeBlockName.lastIndexOf(CarbonCommonConstants.HYPHEN));
   }
 
-  /**
-   * returns segment id from segment name
-   *
-   * @param segmentName
-   * @return
-   */
-  public static String getSegmentId(String segmentName) {
-    return segmentName.split(CarbonCommonConstants.UNDERSCORE)[1];
-  }
-
   public static long getLatestTaskIdForSegment(String segmentId, CarbonTablePath tablePath) {
     String segmentDirPath = tablePath.getCarbonDataDirectoryPath(segmentId);
 
@@ -523,7 +511,7 @@ public class CarbonUpdateUtil {
 
           // aborted scenario.
           invalidDeleteDeltaFiles = updateStatusManager
-              .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, false,
+              .getDeleteDeltaInvalidFilesList(block, false,
                   allSegmentFiles, isAbortedFile);
           for (CarbonFile invalidFile : invalidDeleteDeltaFiles) {
             boolean doForceDelete = true;
@@ -533,7 +521,7 @@ public class CarbonUpdateUtil {
           // case 1
           if (CarbonUpdateUtil.isBlockInvalid(block.getSegmentStatus())) {
             completeListOfDeleteDeltaFiles = updateStatusManager
-                    .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, true,
+                    .getDeleteDeltaInvalidFilesList(block, true,
                             allSegmentFiles, isInvalidFile);
             for (CarbonFile invalidFile : completeListOfDeleteDeltaFiles) {
 
@@ -541,7 +529,7 @@ public class CarbonUpdateUtil {
             }
 
             CarbonFile[] blockRelatedFiles = updateStatusManager
-                    .getAllBlockRelatedFiles(block.getBlockName(), allSegmentFiles,
+                    .getAllBlockRelatedFiles(allSegmentFiles,
                             block.getActualBlockName());
 
             // now for each invalid index file need to check the query execution time out
@@ -555,7 +543,7 @@ public class CarbonUpdateUtil {
 
           } else {
             invalidDeleteDeltaFiles = updateStatusManager
-                    .getDeleteDeltaInvalidFilesList(segment.getLoadName(), block, false,
+                    .getDeleteDeltaInvalidFilesList(block, false,
                             allSegmentFiles, isInvalidFile);
             for (CarbonFile invalidFile : invalidDeleteDeltaFiles) {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java
index 7affe12..20db937 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockDetails.java
@@ -48,14 +48,6 @@ public class DeleteDeltaBlockDetails implements Serializable {
     blockletDetailsMap = new TreeMap<>();
   }
 
-  public String getBlockName() {
-    return blockName;
-  }
-
-  public void setBlockName(String blockName) {
-    this.blockName = blockName;
-  }
-
   @Override public boolean equals(Object obj) {
     if (this == obj) return true;
     if (obj == null || !(obj instanceof DeleteDeltaBlockDetails)) return false;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java
index 22785fa..6185134 100644
--- a/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java
+++ b/core/src/main/java/org/apache/carbondata/core/mutate/DeleteDeltaBlockletDetails.java
@@ -46,14 +46,6 @@ public class DeleteDeltaBlockletDetails implements Serializable {
     return deletedRows.add(row);
   }
 
-  public String getId() {
-    return id;
-  }
-
-  public void setId(String id) {
-    this.id = id;
-  }
-
   public Integer getPageId() {
     return pageId;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java b/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java
deleted file mode 100644
index 3c0dd4f..0000000
--- a/core/src/main/java/org/apache/carbondata/core/mutate/data/BlockletDeleteDeltaCacheLoader.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.mutate.data;
-
-import java.util.Map;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.update.BlockletLevelDeleteDeltaDataCache;
-import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
-
-/**
- * This class is responsible for loading delete delta file cache based on
- * blocklet id of a particular block
- */
-public class BlockletDeleteDeltaCacheLoader implements DeleteDeltaCacheLoaderIntf {
-  private String blockletID;
-  private DataRefNode blockletNode;
-  private AbsoluteTableIdentifier absoluteIdentifier;
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(BlockletDeleteDeltaCacheLoader.class.getName());
-
-  public BlockletDeleteDeltaCacheLoader(String blockletID, DataRefNode blockletNode,
-      AbsoluteTableIdentifier absoluteIdentifier) {
-    this.blockletID = blockletID;
-    this.blockletNode = blockletNode;
-    this.absoluteIdentifier = absoluteIdentifier;
-  }
-
-  /**
-   * This method will load the delete delta cache based on blocklet id of particular block with
-   * the help of SegmentUpdateStatusManager.
-   */
-  public void loadDeleteDeltaFileDataToCache() {
-    SegmentUpdateStatusManager segmentUpdateStatusManager =
-        new SegmentUpdateStatusManager(absoluteIdentifier);
-    Map<Integer, Integer[]> deleteDeltaFileData = null;
-    BlockletLevelDeleteDeltaDataCache deleteDeltaDataCache = null;
-    if (null == blockletNode.getDeleteDeltaDataCache()) {
-      try {
-        deleteDeltaFileData =
-            segmentUpdateStatusManager.getDeleteDeltaDataFromAllFiles(blockletID);
-        deleteDeltaDataCache = new BlockletLevelDeleteDeltaDataCache(deleteDeltaFileData,
-            segmentUpdateStatusManager.getTimestampForRefreshCache(blockletID, null));
-      } catch (Exception e) {
-        if (LOGGER.isDebugEnabled()) {
-          LOGGER.debug("Unable to retrieve delete delta files");
-        }
-      }
-    } else {
-      deleteDeltaDataCache = blockletNode.getDeleteDeltaDataCache();
-      // if already cache is present then validate the cache using timestamp
-      String cacheTimeStamp = segmentUpdateStatusManager
-          .getTimestampForRefreshCache(blockletID, deleteDeltaDataCache.getCacheTimeStamp());
-      if (null != cacheTimeStamp) {
-        try {
-          deleteDeltaFileData =
-              segmentUpdateStatusManager.getDeleteDeltaDataFromAllFiles(blockletID);
-          deleteDeltaDataCache = new BlockletLevelDeleteDeltaDataCache(deleteDeltaFileData,
-              segmentUpdateStatusManager.getTimestampForRefreshCache(blockletID, cacheTimeStamp));
-        } catch (Exception e) {
-          if (LOGGER.isDebugEnabled()) {
-            LOGGER.debug("Unable to retrieve delete delta files");
-          }
-        }
-      }
-    }
-    blockletNode.setDeleteDeltaDataCache(deleteDeltaDataCache);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java
index b14660c..88b3065 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/ScannedResultCollector.java
@@ -18,7 +18,7 @@ package org.apache.carbondata.core.scan.collector;
 
 import java.util.List;
 
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 
 /**
@@ -27,17 +27,13 @@ import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 public interface ScannedResultCollector {
 
   /**
-   * Below method will be used to aggregate the scanned result
-   *
-   * @param scannedResult scanned result
-   * @return how many records was aggregated
+   * Return the aggregated scanned result in list of rows
    */
-  List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize);
+  List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize);
 
   /**
-   * Collects data in columnar format.
-   * @param scannedResult
-   * @param columnarBatch
+   * Collects result and set it in the specified columnar batch
    */
-  void collectVectorBatch(AbstractScannedResult scannedResult, CarbonColumnarBatch columnarBatch);
+  void collectResultInColumnarBatch(BlockletScannedResult scannedResult,
+      CarbonColumnarBatch columnarBatch);
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
index dc78ac6..694271e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/AbstractScannedResultCollector.java
@@ -19,8 +19,6 @@ package org.apache.carbondata.core.scan.collector.impl;
 import java.math.BigDecimal;
 import java.math.RoundingMode;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
@@ -29,8 +27,8 @@ import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.infos.DimensionInfo;
 import org.apache.carbondata.core.scan.executor.infos.MeasureInfo;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
@@ -39,38 +37,35 @@ import org.apache.carbondata.core.util.DataTypeUtil;
  */
 public abstract class AbstractScannedResultCollector implements ScannedResultCollector {
 
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractScannedResultCollector.class.getName());
-
   /**
    * table block execution infos
    */
-  protected BlockExecutionInfo tableBlockExecutionInfos;
+  BlockExecutionInfo executionInfo;
 
   /**
    * maintains the measure information like datatype, ordinal, measure existence
    */
-  protected MeasureInfo measureInfo;
+  MeasureInfo measureInfo;
 
   /**
    * maintains the dimension information like datatype, ordinal, measure existence
    */
-  protected DimensionInfo dimensionInfo;
+  DimensionInfo dimensionInfo;
 
-  public AbstractScannedResultCollector(BlockExecutionInfo blockExecutionInfos) {
-    this.tableBlockExecutionInfos = blockExecutionInfos;
+  AbstractScannedResultCollector(BlockExecutionInfo blockExecutionInfos) {
+    this.executionInfo = blockExecutionInfos;
     measureInfo = blockExecutionInfos.getMeasureInfo();
     dimensionInfo = blockExecutionInfos.getDimensionInfo();
   }
 
   protected void fillMeasureData(Object[] msrValues, int offset,
-      AbstractScannedResult scannedResult) {
+      BlockletScannedResult scannedResult) {
     int measureExistIndex = 0;
     for (short i = 0; i < measureInfo.getMeasureDataTypes().length; i++) {
       // if measure exists is block then pass measure column
       // data chunk to the collector
       if (measureInfo.getMeasureExists()[i]) {
-        QueryMeasure queryMeasure = tableBlockExecutionInfos.getQueryMeasures()[measureExistIndex];
+        ProjectionMeasure queryMeasure = executionInfo.getProjectionMeasures()[measureExistIndex];
         msrValues[i + offset] = getMeasureData(
             scannedResult.getMeasureChunk(measureInfo.getMeasureOrdinals()[measureExistIndex]),
             scannedResult.getCurrentRowId(), queryMeasure.getMeasure());
@@ -87,8 +82,7 @@ public abstract class AbstractScannedResultCollector implements ScannedResultCol
     }
   }
 
-  protected Object getMeasureData(ColumnPage dataChunk, int index,
-      CarbonMeasure carbonMeasure) {
+  Object getMeasureData(ColumnPage dataChunk, int index, CarbonMeasure carbonMeasure) {
     if (!dataChunk.getNullBits().get(index)) {
       DataType dataType = carbonMeasure.getDataType();
       if (dataType == DataTypes.BOOLEAN) {
@@ -114,7 +108,8 @@ public abstract class AbstractScannedResultCollector implements ScannedResultCol
     return null;
   }
 
-  @Override public void collectVectorBatch(AbstractScannedResult scannedResult,
+  @Override
+  public void collectResultInColumnarBatch(BlockletScannedResult scannedResult,
       CarbonColumnarBatch columnarBatch) {
     throw new UnsupportedOperationException("Works only for batch collectors");
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
index cf6e6d6..4322034 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedResultCollector.java
@@ -29,9 +29,9 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
@@ -42,49 +42,50 @@ import org.apache.commons.lang3.ArrayUtils;
  */
 public class DictionaryBasedResultCollector extends AbstractScannedResultCollector {
 
-  protected QueryDimension[] queryDimensions;
+  protected ProjectionDimension[] queryDimensions;
 
-  protected QueryMeasure[] queryMeasures;
+  protected ProjectionMeasure[] queryMeasures;
 
-  protected DirectDictionaryGenerator[] directDictionaryGenerators;
+  private DirectDictionaryGenerator[] directDictionaryGenerators;
 
   /**
    * query order
    */
   protected int[] order;
 
-  protected int[] actualIndexInSurrogateKey;
+  private int[] actualIndexInSurrogateKey;
 
-  protected boolean[] dictionaryEncodingArray;
+  boolean[] dictionaryEncodingArray;
 
-  protected boolean[] directDictionaryEncodingArray;
+  boolean[] directDictionaryEncodingArray;
 
-  protected boolean[] implictColumnArray;
+  private boolean[] implictColumnArray;
 
-  protected boolean[] complexDataTypeArray;
+  private boolean[] complexDataTypeArray;
 
-  protected int dictionaryColumnIndex;
-  protected int noDictionaryColumnIndex;
-  protected int complexTypeColumnIndex;
+  int dictionaryColumnIndex;
+  int noDictionaryColumnIndex;
+  int complexTypeColumnIndex;
 
-  protected boolean isDimensionExists;
+  boolean isDimensionExists;
 
-  protected Map<Integer, GenericQueryType> comlexDimensionInfoMap;
+  private Map<Integer, GenericQueryType> comlexDimensionInfoMap;
 
   public DictionaryBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
-    queryDimensions = tableBlockExecutionInfos.getQueryDimensions();
-    queryMeasures = tableBlockExecutionInfos.getQueryMeasures();
+    queryDimensions = executionInfo.getProjectionDimensions();
+    queryMeasures = executionInfo.getProjectionMeasures();
     initDimensionAndMeasureIndexesForFillingData();
     isDimensionExists = queryDimensions.length > 0;
-    this.comlexDimensionInfoMap = tableBlockExecutionInfos.getComlexDimensionInfoMap();
+    this.comlexDimensionInfoMap = executionInfo.getComlexDimensionInfoMap();
   }
 
   /**
    * This method will add a record both key and value to list object
    * it will keep track of how many record is processed, to handle limit scenario
    */
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
 
     // scan the record and add to list
     List<Object[]> listBasedResult = new ArrayList<>(batchSize);
@@ -118,13 +119,13 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
     return listBasedResult;
   }
 
-  protected void fillDimensionData(AbstractScannedResult scannedResult, int[] surrogateResult,
+  void fillDimensionData(BlockletScannedResult scannedResult, int[] surrogateResult,
       byte[][] noDictionaryKeys, byte[][] complexTypeKeyArray,
       Map<Integer, GenericQueryType> comlexDimensionInfoMap, Object[] row, int i) {
     if (!dictionaryEncodingArray[i]) {
       if (implictColumnArray[i]) {
         if (CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID
-            .equals(queryDimensions[i].getDimension().getColName())) {
+            .equals(queryDimensions[i].getColumnName())) {
           row[order[i]] = DataTypeUtil.getDataBasedOnDataType(
               scannedResult.getBlockletId() + CarbonCommonConstants.FILE_SEPARATOR + scannedResult
                   .getCurrentPageCounter() + CarbonCommonConstants.FILE_SEPARATOR + scannedResult
@@ -153,7 +154,7 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
     }
   }
 
-  protected void fillMeasureData(AbstractScannedResult scannedResult, Object[] row) {
+  void fillMeasureData(BlockletScannedResult scannedResult, Object[] row) {
     if (measureInfo.getMeasureDataTypes().length > 0) {
       Object[] msrValues = new Object[measureInfo.getMeasureDataTypes().length];
       fillMeasureData(msrValues, 0, scannedResult);
@@ -163,7 +164,7 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
     }
   }
 
-  protected void initDimensionAndMeasureIndexesForFillingData() {
+  void initDimensionAndMeasureIndexesForFillingData() {
     List<Integer> dictionaryIndexes = new ArrayList<Integer>();
     for (int i = 0; i < queryDimensions.length; i++) {
       if (queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY) || queryDimensions[i]
@@ -190,10 +191,10 @@ public class DictionaryBasedResultCollector extends AbstractScannedResultCollect
     complexDataTypeArray = CarbonUtil.getComplexDataTypeArray(queryDimensions);
     order = new int[queryDimensions.length + queryMeasures.length];
     for (int i = 0; i < queryDimensions.length; i++) {
-      order[i] = queryDimensions[i].getQueryOrder();
+      order[i] = queryDimensions[i].getOrdinal();
     }
     for (int i = 0; i < queryMeasures.length; i++) {
-      order[i + queryDimensions.length] = queryMeasures[i].getQueryOrder();
+      order[i + queryDimensions.length] = queryMeasures[i].getOrdinal();
     }
     directDictionaryGenerators = new DirectDictionaryGenerator[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
index 5e6c99a..e6cc817 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/DictionaryBasedVectorResultCollector.java
@@ -23,9 +23,9 @@ import java.util.List;
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
 import org.apache.carbondata.core.scan.result.vector.MeasureDataVectorProcessor;
@@ -35,35 +35,35 @@ import org.apache.carbondata.core.scan.result.vector.MeasureDataVectorProcessor;
  */
 public class DictionaryBasedVectorResultCollector extends AbstractScannedResultCollector {
 
-  protected QueryDimension[] queryDimensions;
+  protected ProjectionDimension[] queryDimensions;
 
-  protected QueryMeasure[] queryMeasures;
+  protected ProjectionMeasure[] queryMeasures;
 
-  protected ColumnVectorInfo[] dictionaryInfo;
+  private ColumnVectorInfo[] dictionaryInfo;
 
-  protected ColumnVectorInfo[] noDictionaryInfo;
+  private ColumnVectorInfo[] noDictionaryInfo;
 
-  protected ColumnVectorInfo[] complexInfo;
+  private ColumnVectorInfo[] complexInfo;
 
-  protected ColumnVectorInfo[] measureColumnInfo;
+  private ColumnVectorInfo[] measureColumnInfo;
 
-  protected ColumnVectorInfo[] allColumnInfo;
+  ColumnVectorInfo[] allColumnInfo;
 
-  protected ColumnVectorInfo[] implictColumnInfo;
+  private ColumnVectorInfo[] implictColumnInfo;
 
   public DictionaryBasedVectorResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
     // initialize only if the current block is not a restructured block else the initialization
     // will be taken care by RestructureBasedVectorResultCollector
     if (!blockExecutionInfos.isRestructuredBlock()) {
-      queryDimensions = tableBlockExecutionInfos.getQueryDimensions();
-      queryMeasures = tableBlockExecutionInfos.getQueryMeasures();
+      queryDimensions = executionInfo.getProjectionDimensions();
+      queryMeasures = executionInfo.getProjectionMeasures();
       allColumnInfo = new ColumnVectorInfo[queryDimensions.length + queryMeasures.length];
       prepareDimensionAndMeasureColumnVectors();
     }
   }
 
-  protected void prepareDimensionAndMeasureColumnVectors() {
+  void prepareDimensionAndMeasureColumnVectors() {
     measureColumnInfo = new ColumnVectorInfo[queryMeasures.length];
     List<ColumnVectorInfo> dictInfoList = new ArrayList<>();
     List<ColumnVectorInfo> noDictInfoList = new ArrayList<>();
@@ -78,13 +78,13 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
         implictColumnList.add(columnVectorInfo);
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       } else if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         noDictInfoList.add(columnVectorInfo);
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         dictInfoList.add(columnVectorInfo);
@@ -92,21 +92,21 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
         columnVectorInfo.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
             .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       } else if (queryDimensions[i].getDimension().isComplex()) {
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         complexList.add(columnVectorInfo);
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
         columnVectorInfo.genericQueryType =
-            tableBlockExecutionInfos.getComlexDimensionInfoMap().get(columnVectorInfo.ordinal);
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+            executionInfo.getComlexDimensionInfoMap().get(columnVectorInfo.ordinal);
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       } else {
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
         dictInfoList.add(columnVectorInfo);
         columnVectorInfo.dimension = queryDimensions[i];
         columnVectorInfo.ordinal = queryDimensions[i].getDimension().getOrdinal();
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       }
     }
     for (int i = 0; i < queryMeasures.length; i++) {
@@ -116,7 +116,7 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
       columnVectorInfo.ordinal = queryMeasures[i].getMeasure().getOrdinal();
       columnVectorInfo.measure = queryMeasures[i];
       this.measureColumnInfo[i] = columnVectorInfo;
-      allColumnInfo[queryMeasures[i].getQueryOrder()] = columnVectorInfo;
+      allColumnInfo[queryMeasures[i].getOrdinal()] = columnVectorInfo;
     }
     dictionaryInfo = dictInfoList.toArray(new ColumnVectorInfo[dictInfoList.size()]);
     noDictionaryInfo = noDictInfoList.toArray(new ColumnVectorInfo[noDictInfoList.size()]);
@@ -126,11 +126,13 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
     Arrays.sort(complexInfo);
   }
 
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
-    throw new UnsupportedOperationException("collectData is not supported here");
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
+    throw new UnsupportedOperationException("collectResultInRow is not supported here");
   }
 
-  @Override public void collectVectorBatch(AbstractScannedResult scannedResult,
+  @Override
+  public void collectResultInColumnarBatch(BlockletScannedResult scannedResult,
       CarbonColumnarBatch columnarBatch) {
     int numberOfPages = scannedResult.numberOfpages();
     int filteredRows = 0;
@@ -150,14 +152,15 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
         return;
       }
       fillColumnVectorDetails(columnarBatch, rowCounter, requiredRows);
-      filteredRows = scannedResult
-          .markFilteredRows(columnarBatch, rowCounter, requiredRows, columnarBatch.getRowCounter());
-      scanAndFillResult(scannedResult, columnarBatch, rowCounter, availableRows, requiredRows);
+      filteredRows = scannedResult.markFilteredRows(
+          columnarBatch, rowCounter, requiredRows, columnarBatch.getRowCounter());
+      fillResultToColumnarBatch(
+          scannedResult, columnarBatch, rowCounter, availableRows, requiredRows);
       columnarBatch.setActualSize(columnarBatch.getActualSize() + requiredRows - filteredRows);
     }
   }
 
-  protected void scanAndFillResult(AbstractScannedResult scannedResult,
+  void fillResultToColumnarBatch(BlockletScannedResult scannedResult,
       CarbonColumnarBatch columnarBatch, int rowCounter, int availableRows, int requiredRows) {
     scannedResult.fillColumnarDictionaryBatch(dictionaryInfo);
     scannedResult.fillColumnarNoDictionaryBatch(noDictionaryInfo);
@@ -174,8 +177,8 @@ public class DictionaryBasedVectorResultCollector extends AbstractScannedResultC
     columnarBatch.setRowCounter(columnarBatch.getRowCounter() + requiredRows);
   }
 
-  protected void fillColumnVectorDetails(CarbonColumnarBatch columnarBatch, int rowCounter,
-      int requiredRows) {
+  void fillColumnVectorDetails(CarbonColumnarBatch columnarBatch, int rowCounter, int requiredRows)
+  {
     for (int i = 0; i < allColumnInfo.length; i++) {
       allColumnInfo[i].size = requiredRows;
       allColumnInfo[i].offset = rowCounter;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java
index a09676a..0780675 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollector.java
@@ -16,14 +16,12 @@
  */
 package org.apache.carbondata.core.scan.collector.impl;
 
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 
 /**
@@ -31,15 +29,11 @@ import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
  */
 public class RawBasedResultCollector extends AbstractScannedResultCollector {
 
-  protected ByteArrayWrapper wrapper;
+  byte[] dictionaryKeyArray;
 
-  protected byte[] dictionaryKeyArray;
+  byte[][] noDictionaryKeyArray;
 
-  protected byte[][] noDictionaryKeyArray;
-
-  protected byte[][] complexTypeKeyArray;
-
-  protected byte[] implicitColumnByteArray;
+  private byte[][] complexTypeKeyArray;
 
   public RawBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
@@ -49,9 +43,10 @@ public class RawBasedResultCollector extends AbstractScannedResultCollector {
    * This method will add a record both key and value to list object
    * it will keep track of how many record is processed, to handle limit scenario
    */
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
     List<Object[]> listBasedResult = new ArrayList<>(batchSize);
-    QueryMeasure[] queryMeasures = tableBlockExecutionInfos.getQueryMeasures();
+    ProjectionMeasure[] queryMeasures = executionInfo.getProjectionMeasures();
     // scan the record and add to list
     int rowCounter = 0;
     while (scannedResult.hasNext() && rowCounter < batchSize) {
@@ -65,24 +60,21 @@ public class RawBasedResultCollector extends AbstractScannedResultCollector {
     return listBasedResult;
   }
 
-  protected void prepareRow(AbstractScannedResult scannedResult, List<Object[]> listBasedResult,
-      QueryMeasure[] queryMeasures) {
+  void prepareRow(BlockletScannedResult scannedResult, List<Object[]> listBasedResult,
+      ProjectionMeasure[] queryMeasures) {
     Object[] row = new Object[1 + queryMeasures.length];
-    wrapper = new ByteArrayWrapper();
+    ByteArrayWrapper wrapper = new ByteArrayWrapper();
     wrapper.setDictionaryKey(dictionaryKeyArray);
     wrapper.setNoDictionaryKeys(noDictionaryKeyArray);
     wrapper.setComplexTypesKeys(complexTypeKeyArray);
-    wrapper.setImplicitColumnByteArray(implicitColumnByteArray);
     row[0] = wrapper;
     fillMeasureData(row, 1, scannedResult);
     listBasedResult.add(row);
   }
 
-  protected void scanResultAndGetData(AbstractScannedResult scannedResult) {
+  void scanResultAndGetData(BlockletScannedResult scannedResult) {
     dictionaryKeyArray = scannedResult.getDictionaryKeyArray();
     noDictionaryKeyArray = scannedResult.getNoDictionaryKeyArray();
     complexTypeKeyArray = scannedResult.getComplexTypeKeyArray();
-    implicitColumnByteArray = scannedResult.getBlockletId()
-        .getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java
index 296d9a6..8b42a4a 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedDictionaryResultCollector.java
@@ -23,8 +23,8 @@ import java.util.Map;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.util.RestructureUtil;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 
 /**
  * class for handling restructure scenarios for filling result
@@ -35,8 +35,8 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe
 
   public RestructureBasedDictionaryResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
-    queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
-    queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures();
+    queryDimensions = executionInfo.getActualQueryDimensions();
+    queryMeasures = executionInfo.getActualQueryMeasures();
     measureDefaultValues = new Object[queryMeasures.length];
     fillMeasureDefaultValues();
     initDimensionAndMeasureIndexesForFillingData();
@@ -61,7 +61,8 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe
    * This method will add a record both key and value to list object
    * it will keep track of how many record is processed, to handle limit scenario
    */
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
     // scan the record and add to list
     List<Object[]> listBasedResult = new ArrayList<>(batchSize);
     int rowCounter = 0;
@@ -69,7 +70,7 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe
     byte[][] noDictionaryKeys;
     byte[][] complexTypeKeyArray;
     Map<Integer, GenericQueryType> comlexDimensionInfoMap =
-        tableBlockExecutionInfos.getComlexDimensionInfoMap();
+        executionInfo.getComlexDimensionInfoMap();
     while (scannedResult.hasNext() && rowCounter < batchSize) {
       Object[] row = new Object[queryDimensions.length + queryMeasures.length];
       if (isDimensionExists) {
@@ -107,13 +108,13 @@ public class RestructureBasedDictionaryResultCollector extends DictionaryBasedRe
   }
 
   protected void fillMeasureData(Object[] msrValues, int offset,
-      AbstractScannedResult scannedResult) {
+      BlockletScannedResult scannedResult) {
     int measureExistIndex = 0;
     for (short i = 0; i < measureInfo.getMeasureDataTypes().length; i++) {
       // if measure exists is block then pass measure column
       // data chunk to the collector
       if (measureInfo.getMeasureExists()[i]) {
-        QueryMeasure queryMeasure = tableBlockExecutionInfos.getQueryMeasures()[measureExistIndex];
+        ProjectionMeasure queryMeasure = executionInfo.getProjectionMeasures()[measureExistIndex];
         msrValues[i + offset] = getMeasureData(
             scannedResult.getMeasureChunk(measureInfo.getMeasureOrdinals()[measureExistIndex]),
             scannedResult.getCurrentRowId(), queryMeasure.getMeasure());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
index b08a1d4..6544a75 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedRawResultCollector.java
@@ -30,9 +30,9 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import org.apache.commons.lang3.ArrayUtils;
@@ -70,15 +70,15 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    */
   private void initRestructuredKeyGenerator() {
     SegmentProperties segmentProperties =
-        tableBlockExecutionInfos.getDataBlock().getSegmentProperties();
-    QueryDimension[] queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
+        executionInfo.getDataBlock().getSegmentProperties();
+    ProjectionDimension[] queryDimensions = executionInfo.getActualQueryDimensions();
     List<Integer> updatedColumnCardinality = new ArrayList<>(queryDimensions.length);
     List<Integer> updatedDimensionPartitioner = new ArrayList<>(queryDimensions.length);
-    int[] dictionaryColumnBlockIndex = tableBlockExecutionInfos.getDictionaryColumnBlockIndex();
+    int[] dictionaryColumnBlockIndex = executionInfo.getDictionaryColumnChunkIndex();
     int dimCounterInCurrentBlock = 0;
     for (int i = 0; i < queryDimensions.length; i++) {
       if (queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        if (tableBlockExecutionInfos.getDimensionInfo().getDimensionExists()[i]) {
+        if (executionInfo.getDimensionInfo().getDimensionExists()[i]) {
           // get the dictionary key ordinal as column cardinality in segment properties
           // will only be for dictionary encoded columns
           CarbonDimension currentBlockDimension = segmentProperties.getDimensions()
@@ -124,8 +124,8 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    */
   private void initCurrentBlockKeyGenerator() {
     SegmentProperties segmentProperties =
-        tableBlockExecutionInfos.getDataBlock().getSegmentProperties();
-    int[] dictionaryColumnBlockIndex = tableBlockExecutionInfos.getDictionaryColumnBlockIndex();
+        executionInfo.getDataBlock().getSegmentProperties();
+    int[] dictionaryColumnBlockIndex = executionInfo.getDictionaryColumnChunkIndex();
     int[] updatedColumnCardinality = new int[dictionaryColumnBlockIndex.length];
     int[] updatedDimensionPartitioner = new int[dictionaryColumnBlockIndex.length];
     for (int i = 0; i < dictionaryColumnBlockIndex.length; i++) {
@@ -149,9 +149,10 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    * This method will add a record both key and value to list object
    * it will keep track of how many record is processed, to handle limit scenario
    */
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
     List<Object[]> listBasedResult = new ArrayList<>(batchSize);
-    QueryMeasure[] queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures();
+    ProjectionMeasure[] queryMeasures = executionInfo.getActualQueryMeasures();
     // scan the record and add to list
     int rowCounter = 0;
     while (scannedResult.hasNext() && rowCounter < batchSize) {
@@ -179,7 +180,7 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    * @return
    */
   private byte[] fillDictionaryKeyArrayWithLatestSchema(byte[] dictionaryKeyArray) {
-    QueryDimension[] actualQueryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
+    ProjectionDimension[] actualQueryDimensions = executionInfo.getActualQueryDimensions();
     int newKeyArrayLength = dimensionInfo.getNewDictionaryColumnCount();
     long[] keyArray = null;
     if (null != updatedCurrentBlockKeyGenerator) {
@@ -222,7 +223,7 @@ public class RestructureBasedRawResultCollector extends RawBasedResultCollector
    * @return
    */
   private byte[][] fillNoDictionaryKeyArrayWithLatestSchema(byte[][] noDictionaryKeyArray) {
-    QueryDimension[] actualQueryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
+    ProjectionDimension[] actualQueryDimensions = executionInfo.getActualQueryDimensions();
     byte[][] noDictionaryKeyArrayWithNewlyAddedColumns =
         new byte[noDictionaryKeyArray.length + dimensionInfo.getNewNoDictionaryColumnCount()][];
     int existingColumnValueIndex = 0;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
index 0f1b9f9..61a2992 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/collector/impl/RestructureBasedVectorResultCollector.java
@@ -26,7 +26,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.util.RestructureUtil;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
@@ -43,8 +43,8 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
 
   public RestructureBasedVectorResultCollector(BlockExecutionInfo blockExecutionInfos) {
     super(blockExecutionInfos);
-    queryDimensions = tableBlockExecutionInfos.getActualQueryDimensions();
-    queryMeasures = tableBlockExecutionInfos.getActualQueryMeasures();
+    queryDimensions = executionInfo.getActualQueryDimensions();
+    queryMeasures = executionInfo.getActualQueryMeasures();
     measureDefaultValues = new Object[queryMeasures.length];
     allColumnInfo = new ColumnVectorInfo[queryDimensions.length + queryMeasures.length];
     createVectorForNewlyAddedDimensions();
@@ -66,7 +66,7 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
           columnVectorInfo.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
               .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
         }
-        allColumnInfo[queryDimensions[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryDimensions[i].getOrdinal()] = columnVectorInfo;
       }
     }
   }
@@ -79,7 +79,7 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
       if (!measureInfo.getMeasureExists()[i]) {
         // add a dummy column vector result collector object
         ColumnVectorInfo columnVectorInfo = new ColumnVectorInfo();
-        allColumnInfo[queryMeasures[i].getQueryOrder()] = columnVectorInfo;
+        allColumnInfo[queryMeasures[i].getOrdinal()] = columnVectorInfo;
         columnVectorInfo.measure = queryMeasures[i];
         measureDefaultValues[i] = getMeasureDefaultValue(queryMeasures[i].getMeasure());
       }
@@ -97,11 +97,13 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
         carbonMeasure.getDefaultValue());
   }
 
-  @Override public List<Object[]> collectData(AbstractScannedResult scannedResult, int batchSize) {
-    throw new UnsupportedOperationException("collectData is not supported here");
+  @Override
+  public List<Object[]> collectResultInRow(BlockletScannedResult scannedResult, int batchSize) {
+    throw new UnsupportedOperationException("collectResultInRow is not supported here");
   }
 
-  @Override public void collectVectorBatch(AbstractScannedResult scannedResult,
+  @Override
+  public void collectResultInColumnarBatch(BlockletScannedResult scannedResult,
       CarbonColumnarBatch columnarBatch) {
     int numberOfPages = scannedResult.numberOfpages();
     while (scannedResult.getCurrentPageCounter() < numberOfPages) {
@@ -124,7 +126,8 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
       fillDataForNonExistingDimensions();
       fillDataForNonExistingMeasures();
       // fill existing dimensions and measures data
-      scanAndFillResult(scannedResult, columnarBatch, rowCounter, availableRows, requiredRows);
+      fillResultToColumnarBatch(
+          scannedResult, columnarBatch, rowCounter, availableRows, requiredRows);
       columnarBatch.setActualSize(columnarBatch.getActualSize() + requiredRows - filteredRows);
     }
   }
@@ -133,11 +136,11 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
    * This method will fill the default values of non existing dimensions in the current block
    */
   private void fillDataForNonExistingDimensions() {
-    for (int i = 0; i < tableBlockExecutionInfos.getActualQueryDimensions().length; i++) {
+    for (int i = 0; i < executionInfo.getActualQueryDimensions().length; i++) {
       if (!dimensionInfo.getDimensionExists()[i]) {
-        int queryOrder = tableBlockExecutionInfos.getActualQueryDimensions()[i].getQueryOrder();
+        int queryOrder = executionInfo.getActualQueryDimensions()[i].getOrdinal();
         CarbonDimension dimension =
-            tableBlockExecutionInfos.getActualQueryDimensions()[i].getDimension();
+            executionInfo.getActualQueryDimensions()[i].getDimension();
         if (dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
           // fill direct dictionary column data
           fillDirectDictionaryData(allColumnInfo[queryOrder].vector, allColumnInfo[queryOrder],
@@ -215,10 +218,10 @@ public class RestructureBasedVectorResultCollector extends DictionaryBasedVector
    * This method will fill the default values of non existing measures in the current block
    */
   private void fillDataForNonExistingMeasures() {
-    for (int i = 0; i < tableBlockExecutionInfos.getActualQueryMeasures().length; i++) {
+    for (int i = 0; i < executionInfo.getActualQueryMeasures().length; i++) {
       if (!measureInfo.getMeasureExists()[i]) {
-        int queryOrder = tableBlockExecutionInfos.getActualQueryMeasures()[i].getQueryOrder();
-        CarbonMeasure measure = tableBlockExecutionInfos.getActualQueryMeasures()[i].getMeasure();
+        int queryOrder = executionInfo.getActualQueryMeasures()[i].getOrdinal();
+        CarbonMeasure measure = executionInfo.getActualQueryMeasures()[i].getMeasure();
         ColumnVectorInfo columnVectorInfo = allColumnInfo[queryOrder];
         CarbonColumnVector vector = columnVectorInfo.vector;
         Object defaultValue = measureDefaultValues[i];

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
index 1fcccca..30dd1dd 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryType.java
@@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
 
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 
 import org.apache.spark.sql.catalyst.util.GenericArrayData;
 import org.apache.spark.sql.types.ArrayType;
@@ -86,7 +86,7 @@ public class ArrayQueryType extends ComplexQueryType implements GenericQueryType
     return new ArrayType(null, true);
   }
 
-  @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder)
+  @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder)
       throws IOException {
     readBlockDataChunk(blockChunkHolder);
     children.fillRequiredBlockData(blockChunkHolder);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
index ee43a10..98f0715 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/ComplexQueryType.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.core.scan.complextypes;
 import java.io.IOException;
 
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 
 public class ComplexQueryType {
   protected String name;
@@ -42,7 +42,7 @@ public class ComplexQueryType {
   protected byte[] copyBlockDataChunk(DimensionRawColumnChunk[] rawColumnChunks,
       int rowNumber, int pageNumber) {
     byte[] data =
-        rawColumnChunks[blockIndex].convertToDimColDataChunk(pageNumber).getChunkData(rowNumber);
+        rawColumnChunks[blockIndex].decodeColumnPage(pageNumber).getChunkData(rowNumber);
     byte[] output = new byte[data.length];
     System.arraycopy(data, 0, output, 0, output.length);
     return output;
@@ -51,10 +51,10 @@ public class ComplexQueryType {
   /*
    * This method will read the block data chunk from the respective block
    */
-  protected void readBlockDataChunk(BlocksChunkHolder blockChunkHolder) throws IOException {
-    if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-      blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+  protected void readBlockDataChunk(RawBlockletColumnChunks blockChunkHolder) throws IOException {
+    if (null == blockChunkHolder.getDimensionRawColumnChunks()[blockIndex]) {
+      blockChunkHolder.getDimensionRawColumnChunks()[blockIndex] = blockChunkHolder.getDataBlock()
+          .readDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
index 978a652..b8aa912 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/PrimitiveQueryType.java
@@ -27,7 +27,7 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionary
 import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.apache.carbondata.core.keygenerator.mdkey.Bits;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.DataTypeUtil;
 
 import org.apache.spark.sql.types.BooleanType$;
@@ -113,7 +113,7 @@ public class PrimitiveQueryType extends ComplexQueryType implements GenericQuery
     }
   }
 
-  @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder)
+  @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder)
       throws IOException {
     readBlockDataChunk(blockChunkHolder);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
index 23a9f81..1d4f141 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/complextypes/StructQueryType.java
@@ -25,7 +25,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 
 import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
 import org.apache.spark.sql.types.DataType;
@@ -106,7 +106,7 @@ public class StructQueryType extends ComplexQueryType implements GenericQueryTyp
     return new StructType(fields);
   }
 
-  @Override public void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder)
+  @Override public void fillRequiredBlockData(RawBlockletColumnChunks blockChunkHolder)
       throws IOException {
     readBlockDataChunk(blockChunkHolder);
 


[08/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
index 94a041a..b74c279 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/AbstractDataFileFooterConverter.java
@@ -378,7 +378,6 @@ public abstract class AbstractDataFileFooterConverter {
       cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
     }
     info.setColumnCardinality(cardinality);
-    info.setNumberOfColumns(segmentInfo.getNum_cols());
     return info;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 52305bd..0cc783e 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -48,10 +48,10 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.datamap.Segment;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
@@ -82,7 +82,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatus;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
@@ -248,16 +248,13 @@ public final class CarbonUtil {
   public static ColumnGroupModel getColGroupModel(int[][] columnGroups) {
     int[] columnSplit = new int[columnGroups.length];
     int noOfColumnStore = columnSplit.length;
-    boolean[] columnarStore = new boolean[noOfColumnStore];
 
     for (int i = 0; i < columnGroups.length; i++) {
       columnSplit[i] = columnGroups[i].length;
-      columnarStore[i] = columnGroups[i].length <= 1;
     }
     ColumnGroupModel colGroupModel = new ColumnGroupModel();
     colGroupModel.setNoOfColumnStore(noOfColumnStore);
     colGroupModel.setColumnSplit(columnSplit);
-    colGroupModel.setColumnarStore(columnarStore);
     colGroupModel.setColumnGroup(columnGroups);
     return colGroupModel;
   }
@@ -418,7 +415,7 @@ public final class CarbonUtil {
     }
   }
 
-  public static int getFirstIndexUsingBinarySearch(DimensionColumnDataChunk dimColumnDataChunk,
+  public static int getFirstIndexUsingBinarySearch(DimensionColumnPage dimColumnDataChunk,
       int low, int high, byte[] compareValue, boolean matchUpLimit) {
     int cmpResult = 0;
     while (high >= low) {
@@ -457,7 +454,7 @@ public final class CarbonUtil {
    * @return the compareValue's range index in the dimColumnDataChunk
    */
   public static int[] getRangeIndexUsingBinarySearch(
-      DimensionColumnDataChunk dimColumnDataChunk, int low, int high, byte[] compareValue) {
+      DimensionColumnPage dimColumnDataChunk, int low, int high, byte[] compareValue) {
 
     int[] rangeIndex = new int[2];
     int cmpResult = 0;
@@ -551,7 +548,7 @@ public final class CarbonUtil {
    * @return index value
    */
   public static int nextLesserValueToTarget(int currentIndex,
-      DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue) {
+      DimensionColumnPage dimColumnDataChunk, byte[] compareValue) {
     while (currentIndex - 1 >= 0
         && dimColumnDataChunk.compareTo(currentIndex - 1, compareValue) >= 0) {
       --currentIndex;
@@ -571,7 +568,7 @@ public final class CarbonUtil {
    * @return index value
    */
   public static int nextGreaterValueToTarget(int currentIndex,
-      DimensionColumnDataChunk dimColumnDataChunk, byte[] compareValue, int numerOfRows) {
+      DimensionColumnPage dimColumnDataChunk, byte[] compareValue, int numerOfRows) {
     while (currentIndex + 1 < numerOfRows
         && dimColumnDataChunk.compareTo(currentIndex + 1, compareValue) <= 0) {
       ++currentIndex;
@@ -934,7 +931,7 @@ public final class CarbonUtil {
     return false;
   }
 
-  public static boolean[] getDictionaryEncodingArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getDictionaryEncodingArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -943,7 +940,7 @@ public final class CarbonUtil {
     return dictionaryEncodingArray;
   }
 
-  public static boolean[] getDirectDictionaryEncodingArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getDirectDictionaryEncodingArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -952,7 +949,7 @@ public final class CarbonUtil {
     return dictionaryEncodingArray;
   }
 
-  public static boolean[] getImplicitColumnArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getImplicitColumnArray(ProjectionDimension[] queryDimensions) {
     boolean[] implicitColumnArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       implicitColumnArray[i] = queryDimensions[i].getDimension().hasEncoding(Encoding.IMPLICIT);
@@ -960,7 +957,7 @@ public final class CarbonUtil {
     return implicitColumnArray;
   }
 
-  public static boolean[] getComplexDataTypeArray(QueryDimension[] queryDimensions) {
+  public static boolean[] getComplexDataTypeArray(ProjectionDimension[] queryDimensions) {
     boolean[] dictionaryEncodingArray = new boolean[queryDimensions.length];
     for (int i = 0; i < queryDimensions.length; i++) {
       dictionaryEncodingArray[i] =
@@ -1008,7 +1005,6 @@ public final class CarbonUtil {
       fileFooter.setColumnInTable(schema);
       SegmentInfo segmentInfo = new SegmentInfo();
       segmentInfo.setColumnCardinality(detailInfo.getDimLens());
-      segmentInfo.setNumberOfColumns(detailInfo.getRowCount());
       fileFooter.setSegmentInfo(segmentInfo);
       return fileFooter;
     }
@@ -1048,7 +1044,7 @@ public final class CarbonUtil {
    * @return
    */
   public static long calculateMetaSize(TableBlockInfo tableBlockInfo) throws IOException {
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();
       long footerPointer = completeBlockLength - 8;
@@ -2183,19 +2179,19 @@ public final class CarbonUtil {
   /**
    * Below method will be used to check filter value is present in the data chunk or not
    * @param filterValues
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param low
    * @param high
    * @param chunkRowIndex
    * @return
    */
   public static int isFilterPresent(byte[][] filterValues,
-      DimensionColumnDataChunk dimensionColumnDataChunk, int low, int high, int chunkRowIndex) {
+      DimensionColumnPage dimensionColumnPage, int low, int high, int chunkRowIndex) {
     int compareResult = 0;
     int mid = 0;
     while (low <= high) {
       mid = (low + high) >>> 1;
-      compareResult = dimensionColumnDataChunk.compareTo(chunkRowIndex, filterValues[mid]);
+      compareResult = dimensionColumnPage.compareTo(chunkRowIndex, filterValues[mid]);
       if (compareResult < 0) {
         high = mid - 1;
       } else if (compareResult > 0) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
index e61b477..d665379 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
@@ -21,7 +21,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
@@ -45,7 +45,7 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
   @Override public DataFileFooter readDataFileFooter(TableBlockInfo tableBlockInfo)
       throws IOException {
     DataFileFooter dataFileFooter = new DataFileFooter();
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();
       long footerPointer = completeBlockLength - 8;
@@ -123,7 +123,7 @@ public class DataFileFooterConverter extends AbstractDataFileFooterConverter {
   }
 
   @Override public List<ColumnSchema> getSchema(TableBlockInfo tableBlockInfo) throws IOException {
-    FileHolder fileReader = null;
+    FileReader fileReader = null;
     List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
     try {
       long completeBlockLength = tableBlockInfo.getBlockLength();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
index 74fd09a..afacc0b 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriter.java
@@ -36,17 +36,6 @@ public interface CarbonDictionaryWriter extends Closeable {
   void write(String value) throws IOException;
 
   /**
-   * write method that accepts one value at a time
-   * This method can be used when data is huge and memory is les. In that
-   * case data can be stored to a file and an iterator can iterate over it and
-   * pass one value at a time
-   *
-   * @param value unique dictionary value
-   * @throws IOException if an I/O error occurs
-   */
-  void write(byte[] value) throws IOException;
-
-  /**
    * write method that accepts list of byte arrays as value
    * This can be used when data is less, then string can be converted
    * to byte array for each value and added to a list

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index 82baccc..53411e9 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -144,7 +144,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    * @param value unique dictionary value
    * @throws IOException if an I/O error occurs
    */
-  @Override public void write(byte[] value) throws IOException {
+  private void write(byte[] value) throws IOException {
     if (isFirstTime) {
       init();
       isFirstTime = false;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
index 8bf8a07..08dd791 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImplTest.java
@@ -87,7 +87,7 @@ public class DictionaryCacheLoaderImplTest {
         return 9999;
       }
     };
-    dictionaryCacheLoader.load(dictionaryInfo, columnIdentifier, 0L, 2L, true);
+    dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true);
     assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 4);
   }
 
@@ -97,7 +97,7 @@ public class DictionaryCacheLoaderImplTest {
         return 10000;
       }
     };
-    dictionaryCacheLoader.load(dictionaryInfo, columnIdentifier, 0L, 2L, true);
+    dictionaryCacheLoader.load(dictionaryInfo, 0L, 2L, true);
     assertEquals(dictionaryInfo.getDictionaryChunks().getSize(), 2);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
deleted file mode 100644
index 5c51c87..0000000
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-
-import org.apache.carbondata.core.datastore.impl.DFSFileHolderImpl;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-public class DFSFileHolderImplUnitTest {
-
-  private static DFSFileHolderImpl dfsFileHolder;
-  private static String fileName;
-  private static String fileNameWithEmptyContent;
-  private static File file;
-  private static File fileWithEmptyContent;
-
-  @BeforeClass public static void setup() {
-    dfsFileHolder = new DFSFileHolderImpl();
-    file = new File("Test.carbondata");
-    fileWithEmptyContent = new File("TestEXception.carbondata");
-
-    if (!file.exists()) try {
-      file.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    if (!fileWithEmptyContent.exists()) try {
-      fileWithEmptyContent.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    try {
-      FileOutputStream of = new FileOutputStream(file, true);
-      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
-      br.write("Hello World");
-      br.close();
-    } catch (Exception e) {
-      e.getMessage();
-    }
-    fileName = file.getAbsolutePath();
-    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
-  }
-
-  @AfterClass public static void tearDown() throws IOException  {
-    file.delete();
-    fileWithEmptyContent.delete();
-    dfsFileHolder.finish();
-  }
-
-  @Test public void testReadByteArray() throws IOException  {
-    byte[] result = dfsFileHolder.readByteArray(fileName, 1);
-    byte[] expected_result = new byte[] { 72 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadByteArrayWithFilePath() throws IOException  {
-    byte[] result = dfsFileHolder.readByteArray(fileName, 2L, 2);
-    byte[] expected_result = { 108, 108 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadLong() throws IOException  {
-    long actualResult = dfsFileHolder.readLong(fileName, 1L);
-    long expectedResult = 7308335519855243122L;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadLongForIoException() throws IOException {
-    dfsFileHolder.readLong(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadIntForIoException() throws IOException{
-    dfsFileHolder.readInt(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadInt() throws IOException  {
-    int actualResult = dfsFileHolder.readInt(fileName, 1L);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileName() throws IOException {
-    int actualResult = dfsFileHolder.readInt(fileName);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testReadIntWithFileNameForIOException() throws IOException {
-    dfsFileHolder.readInt(fileNameWithEmptyContent);
-  }
-
-  @Test public void testDouble() throws IOException  {
-    double actualResult = dfsFileHolder.readDouble(fileName, 1L);
-    double expectedResult = 7.3083355198552433E18;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test(expected = IOException.class)
-  public void testDoubleForIoException() throws IOException {
-    dfsFileHolder.readDouble(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test
-  public void testDoubleForIoExceptionwithUpdateCache() throws IOException {
-    new MockUp<FileSystem>() {
-      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
-          throws IOException {
-        throw new IOException();
-      }
-
-    };
-    dfsFileHolder.readDouble(fileName, 1L);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
new file mode 100644
index 0000000..da61a94
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileReaderImplUnitTest.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+
+import org.apache.carbondata.core.datastore.impl.DFSFileReaderImpl;
+
+import mockit.Mock;
+import mockit.MockUp;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
+public class DFSFileReaderImplUnitTest {
+
+  private static DFSFileReaderImpl dfsFileHolder;
+  private static String fileName;
+  private static String fileNameWithEmptyContent;
+  private static File file;
+  private static File fileWithEmptyContent;
+
+  @BeforeClass public static void setup() {
+    dfsFileHolder = new DFSFileReaderImpl();
+    file = new File("Test.carbondata");
+    fileWithEmptyContent = new File("TestEXception.carbondata");
+
+    if (!file.exists()) try {
+      file.createNewFile();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    if (!fileWithEmptyContent.exists()) try {
+      fileWithEmptyContent.createNewFile();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    try {
+      FileOutputStream of = new FileOutputStream(file, true);
+      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
+      br.write("Hello World");
+      br.close();
+    } catch (Exception e) {
+      e.getMessage();
+    }
+    fileName = file.getAbsolutePath();
+    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
+  }
+
+  @AfterClass public static void tearDown() throws IOException  {
+    file.delete();
+    fileWithEmptyContent.delete();
+    dfsFileHolder.finish();
+  }
+
+  @Test public void testReadByteArray() throws IOException  {
+    byte[] result = dfsFileHolder.readByteArray(fileName, 1);
+    byte[] expected_result = new byte[] { 72 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadByteArrayWithFilePath() throws IOException  {
+    byte[] result = dfsFileHolder.readByteArray(fileName, 2L, 2);
+    byte[] expected_result = { 108, 108 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadLong() throws IOException  {
+    long actualResult = dfsFileHolder.readLong(fileName, 1L);
+    long expectedResult = 7308335519855243122L;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test(expected = IOException.class)
+  public void testReadLongForIoException() throws IOException {
+    dfsFileHolder.readLong(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test(expected = IOException.class)
+  public void testReadIntForIoException() throws IOException{
+    dfsFileHolder.readInt(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test public void testReadInt() throws IOException  {
+    int actualResult = dfsFileHolder.readInt(fileName, 1L);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadIntWithFileName() throws IOException {
+    int actualResult = dfsFileHolder.readInt(fileName);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test(expected = IOException.class)
+  public void testReadIntWithFileNameForIOException() throws IOException {
+    dfsFileHolder.readInt(fileNameWithEmptyContent);
+  }
+
+  @Test public void testDouble() throws IOException  {
+    double actualResult = dfsFileHolder.readDouble(fileName, 1L);
+    double expectedResult = 7.3083355198552433E18;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test(expected = IOException.class)
+  public void testDoubleForIoException() throws IOException {
+    dfsFileHolder.readDouble(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test
+  public void testDoubleForIoExceptionwithUpdateCache() throws IOException {
+    new MockUp<FileSystem>() {
+      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
+          throws IOException {
+        throw new IOException();
+      }
+
+    };
+    dfsFileHolder.readDouble(fileName, 1L);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
deleted file mode 100644
index ed50d63..0000000
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-
-import org.apache.carbondata.core.datastore.impl.FileHolderImpl;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.hamcrest.CoreMatchers.equalTo;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertThat;
-
-public class FileHolderImplUnitTest {
-
-  private static FileHolderImpl fileHolder;
-  private static FileHolderImpl fileHolderWithCapacity;
-  private static String fileName;
-  private static String fileNameWithEmptyContent;
-  private static File file;
-  private static File fileWithEmptyContent;
-
-  @BeforeClass public static void setup() {
-    fileHolder = new FileHolderImpl();
-    fileHolderWithCapacity = new FileHolderImpl(50);
-    file = new File("Test.carbondata");
-    fileWithEmptyContent = new File("TestEXception.carbondata");
-
-    if (!file.exists()) try {
-      file.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    if (!fileWithEmptyContent.exists()) try {
-      fileWithEmptyContent.createNewFile();
-    } catch (IOException e) {
-      e.printStackTrace();
-    }
-    try {
-      FileOutputStream of = new FileOutputStream(file, true);
-      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
-      br.write("Hello World");
-      br.close();
-    } catch (Exception e) {
-      e.getMessage();
-    }
-    fileName = file.getAbsolutePath();
-    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
-  }
-
-  @AfterClass public static void tearDown() throws IOException {
-    file.delete();
-    fileWithEmptyContent.delete();
-    fileHolder.finish();
-  }
-
-  @Test public void testReadByteArray() throws IOException  {
-    byte[] result = fileHolder.readByteArray(fileName, 1);
-    byte[] expected_result = new byte[] { 72 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadByteArrayWithFilePath() throws IOException  {
-    byte[] result = fileHolder.readByteArray(fileName, 2L, 2);
-    byte[] expected_result = { 108, 108 };
-    assertThat(result, is(equalTo(expected_result)));
-  }
-
-  @Test public void testReadLong() throws IOException  {
-    long actualResult = fileHolder.readLong(fileName, 1L);
-    long expectedResult = 7308335519855243122L;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadLongForIoException() throws IOException {
-    fileHolder.readLong(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadIntForIoException() throws IOException {
-    fileHolder.readInt(fileNameWithEmptyContent, 1L);
-  }
-
-  @Test public void testReadInt() throws IOException  {
-    int actualResult = fileHolder.readInt(fileName, 1L);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileName() throws IOException  {
-    int actualResult = fileHolder.readInt(fileName);
-    int expectedResult = 1701604463;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testReadIntWithFileNameForIOException() throws IOException  {
-    fileHolder.readInt(fileNameWithEmptyContent);
-
-  }
-
-  @Test public void testDouble() throws IOException  {
-    double actualResult = fileHolder.readDouble(fileName, 1L);
-    double expectedResult = 7.3083355198552433E18;
-    assertThat(actualResult, is(equalTo(expectedResult)));
-  }
-
-  @Test public void testDoubleForIoException() throws IOException {
-    fileHolder.readDouble(fileNameWithEmptyContent, 1L);
-
-  }
-
-  @Test public void testDoubleForIoExceptionwithUpdateCache() throws Exception {
-    new MockUp<FileSystem>() {
-      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
-          throws IOException {
-        throw new IOException();
-      }
-
-    };
-    try {
-      fileHolder.readDouble(fileName, 1L);
-    } catch (Exception e) {
-      assertNull(e.getMessage());
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
new file mode 100644
index 0000000..a6d3235
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileReaderImplUnitTest.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.carbon.datastorage.filesystem.store.impl;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+
+import org.apache.carbondata.core.datastore.impl.FileReaderImpl;
+
+import mockit.Mock;
+import mockit.MockUp;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+
+public class FileReaderImplUnitTest {
+
+  private static FileReaderImpl fileHolder;
+  private static FileReaderImpl fileHolderWithCapacity;
+  private static String fileName;
+  private static String fileNameWithEmptyContent;
+  private static File file;
+  private static File fileWithEmptyContent;
+
+  @BeforeClass public static void setup() {
+    fileHolder = new FileReaderImpl();
+    fileHolderWithCapacity = new FileReaderImpl(50);
+    file = new File("Test.carbondata");
+    fileWithEmptyContent = new File("TestEXception.carbondata");
+
+    if (!file.exists()) try {
+      file.createNewFile();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    if (!fileWithEmptyContent.exists()) try {
+      fileWithEmptyContent.createNewFile();
+    } catch (IOException e) {
+      e.printStackTrace();
+    }
+    try {
+      FileOutputStream of = new FileOutputStream(file, true);
+      BufferedWriter br = new BufferedWriter(new OutputStreamWriter(of, "UTF-8"));
+      br.write("Hello World");
+      br.close();
+    } catch (Exception e) {
+      e.getMessage();
+    }
+    fileName = file.getAbsolutePath();
+    fileNameWithEmptyContent = fileWithEmptyContent.getAbsolutePath();
+  }
+
+  @AfterClass public static void tearDown() throws IOException {
+    file.delete();
+    fileWithEmptyContent.delete();
+    fileHolder.finish();
+  }
+
+  @Test public void testReadByteArray() throws IOException  {
+    byte[] result = fileHolder.readByteArray(fileName, 1);
+    byte[] expected_result = new byte[] { 72 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadByteArrayWithFilePath() throws IOException  {
+    byte[] result = fileHolder.readByteArray(fileName, 2L, 2);
+    byte[] expected_result = { 108, 108 };
+    assertThat(result, is(equalTo(expected_result)));
+  }
+
+  @Test public void testReadLong() throws IOException  {
+    long actualResult = fileHolder.readLong(fileName, 1L);
+    long expectedResult = 7308335519855243122L;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadLongForIoException() throws IOException {
+    fileHolder.readLong(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test public void testReadIntForIoException() throws IOException {
+    fileHolder.readInt(fileNameWithEmptyContent, 1L);
+  }
+
+  @Test public void testReadInt() throws IOException  {
+    int actualResult = fileHolder.readInt(fileName, 1L);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadIntWithFileName() throws IOException  {
+    int actualResult = fileHolder.readInt(fileName);
+    int expectedResult = 1701604463;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testReadIntWithFileNameForIOException() throws IOException  {
+    fileHolder.readInt(fileNameWithEmptyContent);
+
+  }
+
+  @Test public void testDouble() throws IOException  {
+    double actualResult = fileHolder.readDouble(fileName, 1L);
+    double expectedResult = 7.3083355198552433E18;
+    assertThat(actualResult, is(equalTo(expectedResult)));
+  }
+
+  @Test public void testDoubleForIoException() throws IOException {
+    fileHolder.readDouble(fileNameWithEmptyContent, 1L);
+
+  }
+
+  @Test public void testDoubleForIoExceptionwithUpdateCache() throws Exception {
+    new MockUp<FileSystem>() {
+      @SuppressWarnings("unused") @Mock public FSDataInputStream open(Path file)
+          throws IOException {
+        throw new IOException();
+      }
+
+    };
+    try {
+      fileHolder.readDouble(fileName, 1L);
+    } catch (Exception e) {
+      assertNull(e.getMessage());
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
index 9c43553..20036ec 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/block/SegmentPropertiesTest.java
@@ -84,7 +84,7 @@ public class SegmentPropertiesTest extends TestCase {
     assertTrue(true);
   }
 
-  @Test public void testBlockMetadataHasProperDimensionBlockMapping() {
+  @Test public void testBlockMetadataHasProperDimensionChunkMapping() {
     Map<Integer, Integer> dimensionOrdinalToBlockMapping = new HashMap<Integer, Integer>();
     dimensionOrdinalToBlockMapping.put(0, 0);
     dimensionOrdinalToBlockMapping.put(1, 1);
@@ -97,7 +97,7 @@ public class SegmentPropertiesTest extends TestCase {
     dimensionOrdinalToBlockMapping.put(8, 5);
     dimensionOrdinalToBlockMapping.put(9, 6);
     Map<Integer, Integer> dimensionOrdinalToBlockMappingActual =
-        blockMetadataInfos.getDimensionOrdinalToBlockMapping();
+        blockMetadataInfos.getDimensionOrdinalToChunkMapping();
     assertEquals(dimensionOrdinalToBlockMapping.size(),
         dimensionOrdinalToBlockMappingActual.size());
     Iterator<Entry<Integer, Integer>> iterator =
@@ -112,12 +112,12 @@ public class SegmentPropertiesTest extends TestCase {
     assertTrue(true);
   }
 
-  @Test public void testBlockMetadataHasProperMeasureBlockMapping() {
+  @Test public void testBlockMetadataHasProperMeasureChunkMapping() {
     Map<Integer, Integer> measureOrdinalToBlockMapping = new HashMap<Integer, Integer>();
     measureOrdinalToBlockMapping.put(0, 0);
     measureOrdinalToBlockMapping.put(1, 1);
     Map<Integer, Integer> measureOrdinalToBlockMappingActual =
-        blockMetadataInfos.getMeasuresOrdinalToBlockMapping();
+        blockMetadataInfos.getMeasuresOrdinalToChunkMapping();
     assertEquals(measureOrdinalToBlockMapping.size(), measureOrdinalToBlockMappingActual.size());
     Iterator<Entry<Integer, Integer>> iterator = measureOrdinalToBlockMapping.entrySet().iterator();
     while (iterator.hasNext()) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
index bdb83cd..54b66a6 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/ColumnGroupDimensionDataChunkTest.java
@@ -37,7 +37,7 @@ import org.junit.Test;
 
 public class ColumnGroupDimensionDataChunkTest {
 
-  static ColumnGroupDimensionDataChunk columnGroupDimensionDataChunk;
+  static ColumnGroupDimensionColumnPage columnGroupDimensionDataChunk;
   static KeyGenerator keyGenerator;
 
   @BeforeClass public static void setup() {
@@ -56,7 +56,7 @@ public class ColumnGroupDimensionDataChunkTest {
       position += keyGenerator.getKeySizeInBytes();
     }
     columnGroupDimensionDataChunk =
-        new ColumnGroupDimensionDataChunk(data, keyGenerator.getKeySizeInBytes(), 3);
+        new ColumnGroupDimensionColumnPage(data, keyGenerator.getKeySizeInBytes(), 3);
   }
 
   @Test public void fillChunkDataTest() {
@@ -64,7 +64,7 @@ public class ColumnGroupDimensionDataChunkTest {
     ordinals.add(1);
     KeyStructureInfo keyStructureInfo = getKeyStructureInfo(ordinals, keyGenerator);
     byte[] buffer = new byte[1];
-    columnGroupDimensionDataChunk.fillChunkData(buffer, 0, 1, keyStructureInfo);
+    columnGroupDimensionDataChunk.fillRawData(1, 0, buffer, keyStructureInfo);
     assertEquals(buffer[0], 2);
   }
 
@@ -81,7 +81,7 @@ public class ColumnGroupDimensionDataChunkTest {
     ordinals.add(2);
     KeyStructureInfo keyStructureInfo = getKeyStructureInfo(ordinals, keyGenerator);
     keyStructureInfo.setMdkeyQueryDimensionOrdinal(new int[] { 2 });
-    int res = columnGroupDimensionDataChunk.fillConvertedChunkData(2, 2, row, keyStructureInfo);
+    int res = columnGroupDimensionDataChunk.fillSurrogateKey(2, 2, row, keyStructureInfo);
     Assert.assertTrue(Arrays.equals(row, expected));
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
index 04b7a80..f327ef6 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/chunk/impl/FixedLengthDimensionDataChunkTest.java
@@ -30,7 +30,7 @@ import org.junit.Test;
 
 public class FixedLengthDimensionDataChunkTest {
 
-  static FixedLengthDimensionDataChunk fixedLengthDimensionDataChunk;
+  static FixedLengthDimensionColumnPage fixedLengthDimensionDataChunk;
   static byte[] data;
 
   @BeforeClass public static void setup() {
@@ -40,7 +40,7 @@ public class FixedLengthDimensionDataChunkTest {
 
     int invertedIndexReverse[] = { 1, 0, 5, 7, 8 };
     fixedLengthDimensionDataChunk =
-        new FixedLengthDimensionDataChunk(data, invertedIndex, invertedIndexReverse, 5, 4);
+        new FixedLengthDimensionColumnPage(data, invertedIndex, invertedIndexReverse, 5, 4);
   }
 
   @Test public void fillChunkDataTest() {
@@ -48,7 +48,7 @@ public class FixedLengthDimensionDataChunkTest {
     int[] maskByteRanges = { 1, 2, 4, 6, 5 };
     keyStructureInfo.setMaskByteRanges(maskByteRanges);
     keyStructureInfo.setMaxKey("1234567".getBytes());
-    int res = fixedLengthDimensionDataChunk.fillChunkData(data, 0, 0, keyStructureInfo);
+    int res = fixedLengthDimensionDataChunk.fillRawData(0, 0, data, keyStructureInfo);
     int expectedResult = 4 ;
     assertEquals(res, expectedResult);
   }
@@ -62,7 +62,7 @@ public class FixedLengthDimensionDataChunkTest {
   @Test public void fillConvertedChunkDataTest() {
     int[] row = { 1, 2, 4, 6 };
     KeyStructureInfo keyStructureInfo = new KeyStructureInfo();
-    int res = fixedLengthDimensionDataChunk.fillConvertedChunkData(1, 0, row, keyStructureInfo);
+    int res = fixedLengthDimensionDataChunk.fillSurrogateKey(1, 0, row, keyStructureInfo);
     int expectedResult = 1;
     assertEquals(res, expectedResult);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
index d874037..7f7719c 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/impl/btree/BTreeBlockFinderTest.java
@@ -76,9 +76,9 @@ public class BTreeBlockFinderTest extends TestCase {
     buffer.array();
     IndexKey key = new IndexKey(null, buffer.array());
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(1, findFirstBlock.nodeNumber());
+    assertEquals(1, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(1, findLastBlock.nodeNumber());
+    assertEquals(1, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithNoDictionary() {
@@ -96,9 +96,9 @@ public class BTreeBlockFinderTest extends TestCase {
     buffer.array();
     IndexKey key = new IndexKey(null, buffer.array());
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithDictionaryKey1()
@@ -118,9 +118,9 @@ public class BTreeBlockFinderTest extends TestCase {
     IndexKey key =
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 1, 1 }), null);
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   @Test public void testBtreeSearchIsWorkingAndGivingPorperBlockletWithDictionaryKey2()
@@ -141,9 +141,9 @@ public class BTreeBlockFinderTest extends TestCase {
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 0, 0 }), null);
 
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(0, findFirstBlock.nodeNumber());
+    assertEquals(0, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(0, findLastBlock.nodeNumber());
+    assertEquals(0, findLastBlock.nodeIndex());
   }
 
   /**
@@ -169,9 +169,9 @@ public class BTreeBlockFinderTest extends TestCase {
         new IndexKey(multiDimKeyVarLengthGenerator.generateKey(new int[] { 10001, 10001 }), null);
 
     DataRefNode findFirstBlock = finder.findFirstDataBlock(dataBlock, key);
-    assertEquals(99, findFirstBlock.nodeNumber());
+    assertEquals(99, findFirstBlock.nodeIndex());
     DataRefNode findLastBlock = finder.findLastDataBlock(dataBlock, key);
-    assertEquals(99, findLastBlock.nodeNumber());
+    assertEquals(99, findLastBlock.nodeIndex());
   }
 
   private List<DataFileFooter> getDataFileFooterList() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
index 85bdfd8..be91410 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/collector/impl/RawBasedResultCollectorTest.java
@@ -50,32 +50,32 @@ public class RawBasedResultCollectorTest {
 //    aggregatorInfo.setDefaultValues(new Object[] { 1, 2, 3, 4 });
 //    aggregatorInfo.setMeasureDataTypes(
 //        new DataType[] { DataTypes.INT, DataTypes.TIMESTAMP, DataTypes.INT, DataTypes.INT });
-//    QueryMeasure queryMeasure1 = new QueryMeasure("QMCol1");
+//    ProjectionMeasure queryMeasure1 = new ProjectionMeasure("QMCol1");
 //    queryMeasure1.setQueryOrder(1);
-//    QueryMeasure queryMeasure2 = new QueryMeasure("QMCol2");
+//    ProjectionMeasure queryMeasure2 = new ProjectionMeasure("QMCol2");
 //    queryMeasure1.setQueryOrder(2);
-//    QueryMeasure queryMeasure3 = new QueryMeasure("QMCol3");
+//    ProjectionMeasure queryMeasure3 = new ProjectionMeasure("QMCol3");
 //    queryMeasure1.setQueryOrder(3);
-//    QueryMeasure queryMeasure4 = new QueryMeasure("QMCol4");
+//    ProjectionMeasure queryMeasure4 = new ProjectionMeasure("QMCol4");
 //    queryMeasure1.setQueryOrder(4);
-//    QueryDimension queryDimension1 = new QueryDimension("QDCol1");
+//    ProjectionDimension queryDimension1 = new ProjectionDimension("QDCol1");
 //    queryDimension1.setQueryOrder(1);
 //    ColumnSchema columnSchema = new ColumnSchema();
 //    queryDimension1.setDimension(new CarbonDimension(columnSchema, 0, 0, 0, 0));
-//    QueryDimension queryDimension2 = new QueryDimension("QDCol2");
+//    ProjectionDimension queryDimension2 = new ProjectionDimension("QDCol2");
 //    queryDimension2.setQueryOrder(2);
 //    queryDimension2.setDimension(new CarbonDimension(columnSchema, 1, 1, 1, 1));
-//    QueryDimension queryDimension3 = new QueryDimension("QDCol3");
+//    ProjectionDimension queryDimension3 = new ProjectionDimension("QDCol3");
 //    queryDimension3.setQueryOrder(3);
 //    queryDimension3.setDimension(new CarbonDimension(columnSchema, 2, 0, 0, 0));
-//    QueryDimension queryDimension4 = new QueryDimension("QDCol4");
+//    ProjectionDimension queryDimension4 = new ProjectionDimension("QDCol4");
 //    queryDimension4.setQueryOrder(4);
 //    queryDimension4.setDimension(new CarbonDimension(columnSchema, 3, 0, 0, 0));
-//    blockExecutionInfo.setQueryDimensions(
-//        new QueryDimension[] { queryDimension1, queryDimension2, queryDimension3,
+//    blockExecutionInfo.setProjectionDimensions(
+//        new ProjectionDimension[] { queryDimension1, queryDimension2, queryDimension3,
 //            queryDimension4 });
-//    blockExecutionInfo.setQueryMeasures(
-//        new QueryMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 });
+//    blockExecutionInfo.setProjectionMeasures(
+//        new ProjectionMeasure[] { queryMeasure1, queryMeasure2, queryMeasure3, queryMeasure4 });
 //    blockExecutionInfo.setFixedKeyUpdateRequired(true);
 //    blockExecutionInfo.setMeasureInfo(aggregatorInfo);
 //    blockExecutionInfo.setMaskedByteForBlock(new int[] { 1, 2 });
@@ -99,7 +99,7 @@ public class RawBasedResultCollectorTest {
 //        return new byte[][] { { 1, 2 }, { 1, 2 } };
 //      }
 //
-//      @SuppressWarnings("unused") @Mock public ColumnPage getMeasureChunk(int ordinal) {
+//      @SuppressWarnings("unused") @Mock public ColumnPage readMeasureChunk(int ordinal) {
 //        ColumnPage ColumnPage = new ColumnPage();
 //        PresenceMeta presenceMeta = new PresenceMeta();
 //        BitSet bitSet = new BitSet();
@@ -120,10 +120,10 @@ public class RawBasedResultCollectorTest {
 //      }
 //    };
 //
-//    AbstractScannedResult abstractScannedResult =
+//    BlockletScannedResult abstractScannedResult =
 //        new NonFilterQueryScannedResult(blockExecutionInfo);
-//    abstractScannedResult.setNumberOfRows(2);
-//    List<Object[]> result = rawBasedResultCollector.collectData(abstractScannedResult, 2);
+//    abstractScannedResult.setPageFilteredRowCount(2);
+//    List<Object[]> result = rawBasedResultCollector.collectResultInRow(abstractScannedResult, 2);
 //    int expectedResult = 2;
 //    assertThat(result.size(), is(equalTo(expectedResult)));
 //  }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
index 45e86f2..e8b75b7 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/QueryUtilTest.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.SegmentPropertiesTestUtil;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.scan.model.QueryDimension;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
 
 import junit.framework.TestCase;
 import mockit.Mock;
@@ -45,9 +45,8 @@ public class QueryUtilTest extends TestCase {
 
   @Test public void testGetMaskedByteRangeGivingProperMaksedByteRange() {
 
-    QueryDimension dimension =
-        new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
+    ProjectionDimension dimension =
+        new ProjectionDimension(segmentProperties.getDimensions().get(0));
     int[] maskedByteRange = QueryUtil
         .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator());
     int[] expectedMaskedByteRange = { 0 };
@@ -57,11 +56,10 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaskedByteRangeGivingProperMaksedByteRangeOnlyForDictionaryKey() {
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> dimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       dimensions.add(dimension);
     }
     int[] maskedByteRange =
@@ -84,11 +82,10 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaxKeyBasedOnDimensions() {
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> dimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       dimensions.add(dimension);
     }
     byte[] maxKeyBasedOnDimensions = null;
@@ -115,10 +112,8 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetMaksedByte() {
-    QueryDimension dimension =
-        new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
-    dimension.setDimension(segmentProperties.getDimensions().get(0));
+    ProjectionDimension dimension =
+        new ProjectionDimension(segmentProperties.getDimensions().get(0));
     int[] maskedByteRange = QueryUtil
         .getMaskedByteRange(Arrays.asList(dimension), segmentProperties.getDimensionKeyGenerator());
     int[] maskedByte = QueryUtil
@@ -203,18 +198,16 @@ public class QueryUtilTest extends TestCase {
   }
 
   @Test public void testGetSortDimensionIndexes() {
-    List<QueryDimension> sortedDimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> sortedDimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       sortedDimensions.add(dimension);
     }
-    List<QueryDimension> queryDimensions = new ArrayList<QueryDimension>();
+    List<ProjectionDimension> queryDimensions = new ArrayList<ProjectionDimension>();
     for (int i = 0; i < 2; i++) {
-      QueryDimension dimension =
-          new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      ProjectionDimension dimension =
+          new ProjectionDimension(segmentProperties.getDimensions().get(i));
       queryDimensions.add(dimension);
     }
     byte[] actualValue = QueryUtil.getSortDimensionIndexes(sortedDimensions, queryDimensions);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
index 172a53e..bdf81e7 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/executor/util/RestructureUtilTest.java
@@ -28,8 +28,8 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.infos.MeasureInfo;
-import org.apache.carbondata.core.scan.model.QueryDimension;
-import org.apache.carbondata.core.scan.model.QueryMeasure;
+import org.apache.carbondata.core.scan.model.ProjectionDimension;
+import org.apache.carbondata.core.scan.model.ProjectionMeasure;
 
 import org.junit.Test;
 
@@ -79,22 +79,20 @@ public class RestructureUtilTest {
     List<CarbonDimension> tableComplexDimensions =
         Arrays.asList(tableComplexDimension1, tableComplexDimension2);
 
-    QueryDimension queryDimension1 = new QueryDimension("Id");
-    queryDimension1.setDimension(tableBlockDimension1);
-    QueryDimension queryDimension2 = new QueryDimension("Name");
-    queryDimension2.setDimension(tableComplexDimension2);
-    QueryDimension queryDimension3 = new QueryDimension("Address");
-    queryDimension3.setDimension(new CarbonDimension(columnSchema5, 3, 3, 3, 3));
+    ProjectionDimension queryDimension1 = new ProjectionDimension(tableBlockDimension1);
+    ProjectionDimension queryDimension2 = new ProjectionDimension(tableComplexDimension2);
+    ProjectionDimension
+        queryDimension3 = new ProjectionDimension(new CarbonDimension(columnSchema5, 3, 3, 3, 3));
 
-    List<QueryDimension> queryDimensions =
+    List<ProjectionDimension> queryDimensions =
         Arrays.asList(queryDimension1, queryDimension2, queryDimension3);
 
-    List<QueryDimension> result = null;
+    List<ProjectionDimension> result = null;
     result = RestructureUtil
         .createDimensionInfoAndGetCurrentBlockQueryDimension(blockExecutionInfo, queryDimensions,
             tableBlockDimensions, tableComplexDimensions);
     List<CarbonDimension> resultDimension = new ArrayList<>(result.size());
-    for (QueryDimension queryDimension : result) {
+    for (ProjectionDimension queryDimension : result) {
       resultDimension.add(queryDimension.getDimension());
     }
     assertThat(resultDimension,
@@ -121,13 +119,10 @@ public class RestructureUtilTest {
     carbonMeasure3.getColumnSchema().setDefaultValue("3".getBytes());
     List<CarbonMeasure> currentBlockMeasures = Arrays.asList(carbonMeasure1, carbonMeasure2);
 
-    QueryMeasure queryMeasure1 = new QueryMeasure("Id");
-    queryMeasure1.setMeasure(carbonMeasure1);
-    QueryMeasure queryMeasure2 = new QueryMeasure("Name");
-    queryMeasure2.setMeasure(carbonMeasure2);
-    QueryMeasure queryMeasure3 = new QueryMeasure("Age");
-    queryMeasure3.setMeasure(carbonMeasure3);
-    List<QueryMeasure> queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2, queryMeasure3);
+    ProjectionMeasure queryMeasure1 = new ProjectionMeasure(carbonMeasure1);
+    ProjectionMeasure queryMeasure2 = new ProjectionMeasure(carbonMeasure2);
+    ProjectionMeasure queryMeasure3 = new ProjectionMeasure(carbonMeasure3);
+    List<ProjectionMeasure> queryMeasures = Arrays.asList(queryMeasure1, queryMeasure2, queryMeasure3);
     BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
     RestructureUtil.createMeasureInfoAndGetCurrentBlockQueryMeasures(blockExecutionInfo, queryMeasures,
         currentBlockMeasures);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
index 565da04..9c7c26c 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/FilterUtilTest.java
@@ -328,7 +328,7 @@ public class FilterUtilTest extends AbstractDictionaryCacheTest {
         return "test";
       }
     };
-    assertTrue(FilterUtil.getFilterListForRS(expression, columnExpression, defaultValues,
+    assertTrue(FilterUtil.getFilterListForRS(expression, defaultValues,
         defaultSurrogate) instanceof ColumnFilterInfo);
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
index e3ae42c..11c4980 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/ExcludeFilterExecuterImplTest.java
@@ -18,12 +18,12 @@ package org.apache.carbondata.core.scan.filter.executer;
 
 import java.util.BitSet;
 
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 public class ExcludeFilterExecuterImplTest extends IncludeFilterExecuterImplTest {
 
- @Override public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimColumnDataChunk,
+ @Override public BitSet setFilterdIndexToBitSetNew(DimensionColumnPage dimColumnDataChunk,
      int numerOfRows, byte[][] filterValues) {
    BitSet bitSet = new BitSet(numerOfRows);
    bitSet.flip(0, numerOfRows);
@@ -46,7 +46,7 @@ public class ExcludeFilterExecuterImplTest extends IncludeFilterExecuterImplTest
    return bitSet;
  }
 
- @Override public BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimColumnDataChunk,
+ @Override public BitSet setFilterdIndexToBitSet(DimensionColumnPage dimColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     bitSet.flip(0, numerOfRows);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
index 29dda52..1bfa875 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/filter/executer/IncludeFilterExecuterImplTest.java
@@ -18,8 +18,8 @@ package org.apache.carbondata.core.scan.filter.executer;
 
 import java.util.BitSet;
 
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionColumnPage;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import org.junit.Assert;
@@ -38,22 +38,22 @@ public class IncludeFilterExecuterImplTest extends TestCase {
 
   }
 
-  public BitSet setFilterdIndexToBitSetNew(DimensionColumnDataChunk dimensionColumnDataChunk,
+  public BitSet setFilterdIndexToBitSetNew(DimensionColumnPage dimensionColumnPage,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
-    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+    if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) {
       // byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
       if (filterValues.length > 1) {
         for (int i = 0; i < numerOfRows; i++) {
           int index = CarbonUtil.binarySearch(filterValues, 0, filterValues.length - 1,
-              dimensionColumnDataChunk.getChunkData(i));
+              dimensionColumnPage.getChunkData(i));
           if (index >= 0) {
             bitSet.set(i);
           }
         }
       } else if (filterValues.length == 1) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (dimensionColumnDataChunk.compareTo(i, filterValues[0]) == 0) {
+          if (dimensionColumnPage.compareTo(i, filterValues[0]) == 0) {
             bitSet.set(i);
           }
         }
@@ -62,14 +62,14 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     return bitSet;
   }
 
-  public BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows,
+  public BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage, int numerOfRows,
       byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
-    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+    if (dimensionColumnPage instanceof FixedLengthDimensionColumnPage) {
       // byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
       for (int k = 0; k < filterValues.length; k++) {
         for (int j = 0; j < numerOfRows; j++) {
-          if (dimensionColumnDataChunk.compareTo(j, filterValues[k]) == 0) {
+          if (dimensionColumnPage.compareTo(j, filterValues[k]) == 0) {
             bitSet.set(j);
           }
         }
@@ -163,7 +163,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     System.out.println("dimColumnSize: " + dimColumnSize);
     
-    FixedLengthDimensionDataChunk dimensionColumnDataChunk;
+    FixedLengthDimensionColumnPage dimensionColumnDataChunk;
     DimColumnExecuterFilterInfo dim = new DimColumnExecuterFilterInfo();
 
     byte[] dataChunk = new byte[dataChunkSize * dimColumnSize];
@@ -183,7 +183,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     dim.setFilterKeys(filterKeys);
 
-    dimensionColumnDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    dimensionColumnDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunkSize, dimColumnSize);
 
     // repeat query and compare 2 result between old code and new optimized code
@@ -194,7 +194,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
       oldTime = oldTime + System.currentTimeMillis() - start;
 
       start = System.currentTimeMillis();
-      BitSet bitNew = this.setFilterdIndexToBitSetNew((FixedLengthDimensionDataChunk) dimensionColumnDataChunk, dataChunkSize,
+      BitSet bitNew = this.setFilterdIndexToBitSetNew((FixedLengthDimensionColumnPage) dimensionColumnDataChunk, dataChunkSize,
           filterKeys);
       newTime = newTime + System.currentTimeMillis() - start;
 
@@ -212,7 +212,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
   }
 
 
-  private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSetWithColumnIndexOld(FixedLengthDimensionColumnPage dimensionColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
@@ -243,7 +243,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     return bitSet;
   }
 
-  private BitSet setFilterdIndexToBitSetWithColumnIndexNew(FixedLengthDimensionDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSetWithColumnIndexNew(FixedLengthDimensionColumnPage dimensionColumnDataChunk,
       int numerOfRows, byte[][] filterValues) {
     BitSet bitSet = new BitSet(numerOfRows);
     int startIndex = 0;
@@ -281,7 +281,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     int filteredValueCnt = 800;
     // column dictionary size
     int dimColumnSize = 2;
-    FixedLengthDimensionDataChunk dimensionColumnDataChunk;
+    FixedLengthDimensionColumnPage dimensionColumnDataChunk;
     DimColumnExecuterFilterInfo dim = new DimColumnExecuterFilterInfo();
 
     byte[] dataChunk = new byte[dataChunkSize * dimColumnSize];
@@ -303,7 +303,7 @@ public class IncludeFilterExecuterImplTest extends TestCase {
     }
     dim.setFilterKeys(filterKeys);
 
-    dimensionColumnDataChunk = new FixedLengthDimensionDataChunk(dataChunk, null, null,
+    dimensionColumnDataChunk = new FixedLengthDimensionColumnPage(dataChunk, null, null,
         dataChunk.length / dimColumnSize, dimColumnSize);
 
     // initial to run

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
deleted file mode 100644
index 541ce48..0000000
--- a/core/src/test/java/org/apache/carbondata/core/scan/result/BatchResultTest.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.result;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import mockit.Mock;
-import mockit.MockUp;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
-public class BatchResultTest {
-  private static BatchResult batchResult;
-  private static List<Object[]> rowsList = new ArrayList(2);
-
-  @BeforeClass public static void setUp() {
-    batchResult = new BatchResult();
-    rowsList.add(0, new Integer[] { 1, 2 });
-    rowsList.add(1, new Integer[] { 3 });
-  }
-
-  @Test public void testNext() throws NoSuchElementException {
-    BatchResult rows = new BatchResult();
-    rows.setRows(rowsList);
-    Object[] result = rows.next();
-    Assert.assertTrue(result.equals(rowsList.get(0)));
-  }
-
-  @Test(expected = NoSuchElementException.class) public void testNextWithNoSuchElementException() {
-    BatchResult rows = new BatchResult();
-    List emptyList = new ArrayList(2);
-    rows.setRows(emptyList);
-    rows.next();
-  }
-
-  @Test public void testGetRows() {
-    new MockUp<BatchResult>() {
-      @Mock public void $init() {
-        //to be left blank
-      }
-    };
-    BatchResult batchResult = new BatchResult();
-    List<Object[]> list = batchResult.getRows();
-    assertNull("Number of rows is null", list);
-  }
-
-  @Test public void testHasNext() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    list.add(1, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    boolean result = batchResult.hasNext();
-    Assert.assertTrue(result);
-  }
-
-  @Test public void testGetRawRow() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    Object[] actualValue = batchResult.getRawRow(0);
-    Assert.assertTrue(list.get(0) == actualValue);
-  }
-
-  @Test public void testGetSize() {
-    List<Object[]> list = new ArrayList<>();
-    list.add(0, new Integer[] { 1, 2 });
-    list.add(1, new Integer[] { 1, 2 });
-    batchResult.setRows(list);
-    int actualValue = batchResult.getSize();
-    int expectedValue = 2;
-    assertEquals(expectedValue, actualValue);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java b/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
new file mode 100644
index 0000000..123d64e
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/scan/result/RowBatchTest.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.result;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import mockit.Mock;
+import mockit.MockUp;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class RowBatchTest {
+  private static RowBatch rowBatch;
+  private static List<Object[]> rowsList = new ArrayList(2);
+
+  @BeforeClass public static void setUp() {
+    rowBatch = new RowBatch();
+    rowsList.add(0, new Integer[] { 1, 2 });
+    rowsList.add(1, new Integer[] { 3 });
+  }
+
+  @Test public void testNext() throws NoSuchElementException {
+    RowBatch rows = new RowBatch();
+    rows.setRows(rowsList);
+    Object[] result = rows.next();
+    Assert.assertTrue(result.equals(rowsList.get(0)));
+  }
+
+  @Test(expected = NoSuchElementException.class) public void testNextWithNoSuchElementException() {
+    RowBatch rows = new RowBatch();
+    List emptyList = new ArrayList(2);
+    rows.setRows(emptyList);
+    rows.next();
+  }
+
+  @Test public void testGetRows() {
+    new MockUp<RowBatch>() {
+      @Mock public void $init() {
+        //to be left blank
+      }
+    };
+    RowBatch rowBatch = new RowBatch();
+    List<Object[]> list = rowBatch.getRows();
+    assertNull("Number of rows is null", list);
+  }
+
+  @Test public void testHasNext() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    list.add(1, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    boolean result = rowBatch.hasNext();
+    Assert.assertTrue(result);
+  }
+
+  @Test public void testGetRawRow() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    Object[] actualValue = rowBatch.getRawRow(0);
+    Assert.assertTrue(list.get(0) == actualValue);
+  }
+
+  @Test public void testGetSize() {
+    List<Object[]> list = new ArrayList<>();
+    list.add(0, new Integer[] { 1, 2 });
+    list.add(1, new Integer[] { 1, 2 });
+    rowBatch.setRows(list);
+    int actualValue = rowBatch.getSize();
+    int expectedValue = 2;
+    assertEquals(expectedValue, actualValue);
+  }
+
+}


[27/49] carbondata git commit: [REBASE] Solve conflict after rebasing master

Posted by ja...@apache.org.
[REBASE] Solve conflict after rebasing master


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

Branch: refs/heads/carbonstore-rebase4
Commit: 806b98410b6a149be513fa136878e031e6b45920
Parents: 2d77936
Author: Jacky Li <ja...@qq.com>
Authored: Fri Feb 9 01:39:20 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 00:30:22 2018 +0800

----------------------------------------------------------------------
 .../scan/filter/FilterExpressionProcessor.java  |  2 +-
 .../filter/executer/FalseFilterExecutor.java    | 16 +++++++-------
 .../FalseConditionalResolverImpl.java           |  4 ++--
 .../apache/carbondata/core/util/CarbonUtil.java |  1 -
 .../core/util/path/CarbonTablePath.java         |  6 +++---
 .../spark/rdd/AggregateDataMapCompactor.scala   | 12 +++++------
 .../preaaggregate/PreAggregateListeners.scala   | 22 +++++++++-----------
 .../CarbonAlterTableDataTypeChangeCommand.scala | 17 ++++++++-------
 .../schema/CarbonAlterTableRenameCommand.scala  |  3 ++-
 .../apache/spark/sql/hive/CarbonMetaStore.scala | 12 ++++++-----
 .../processing/util/CarbonLoaderUtil.java       |  5 +++--
 11 files changed, 51 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
index b882b51..26b202f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/FilterExpressionProcessor.java
@@ -398,7 +398,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
     ConditionalExpression condExpression = null;
     switch (filterExpressionType) {
       case FALSE:
-        return new FalseConditionalResolverImpl(expression, false, false, tableIdentifier);
+        return new FalseConditionalResolverImpl(expression, false, false);
       case TRUE:
         return new TrueConditionalResolverImpl(expression, false, false);
       case EQUALS:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
index 2d2a15c..75a6ec3 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/FalseFilterExecutor.java
@@ -21,7 +21,7 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
@@ -33,8 +33,8 @@ import org.apache.carbondata.core.util.BitSetGroup;
 public class FalseFilterExecutor implements FilterExecuter {
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blocksChunkHolder, boolean useBitsetPipeline)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks blocksChunkHolder,
+      boolean useBitsetPipeline) throws FilterUnsupportedException, IOException {
     int numberOfPages = blocksChunkHolder.getDataBlock().numberOfPages();
     BitSetGroup group = new BitSetGroup(numberOfPages);
     for (int i = 0; i < numberOfPages; i++) {
@@ -44,17 +44,19 @@ public class FalseFilterExecutor implements FilterExecuter {
     return group;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException, IOException {
     return false;
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     return new BitSet();
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks blockChunkHolder) {
     // Do Nothing
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java
index eccda1e..63ce790 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/FalseConditionalResolverImpl.java
@@ -28,8 +28,8 @@ public class FalseConditionalResolverImpl extends ConditionalFilterResolverImpl
   private static final long serialVersionUID = 4599541011924324041L;
 
   public FalseConditionalResolverImpl(Expression exp, boolean isExpressionResolve,
-      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
-    super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false);
+      boolean isIncludeFilter) {
+    super(exp, isExpressionResolve, isIncludeFilter, false);
   }
 
   @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index d531b5c..5ec0158 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -91,7 +91,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 import org.apache.carbondata.core.util.comparator.Comparator;
 import org.apache.carbondata.core.util.comparator.SerializableComparator;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.format.BlockletHeader;
 import org.apache.carbondata.format.DataChunk2;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
index 0164151..238d92a 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
@@ -201,11 +201,11 @@ public class CarbonTablePath {
     return getMetadataPath(tablePath) + File.separator + TABLE_STATUS_FILE;
   }
 
-  public String getTableStatusFilePathWithUUID(String uuid) {
+  public static String getTableStatusFilePathWithUUID(String tablePath, String uuid) {
     if (!uuid.isEmpty()) {
-      return getTableStatusFilePath() + CarbonCommonConstants.UNDERSCORE + uuid;
+      return getTableStatusFilePath(tablePath) + CarbonCommonConstants.UNDERSCORE + uuid;
     } else {
-      return getTableStatusFilePath();
+      return getTableStatusFilePath(tablePath);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
index 184bf1b..8bbe816 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/AggregateDataMapCompactor.scala
@@ -84,8 +84,8 @@ class AggregateDataMapCompactor(carbonLoadModel: CarbonLoadModel,
           case other => other
         }
         SegmentStatusManager.writeLoadDetailsIntoFile(
-          CarbonTablePath.getTableStatusFilePathWithUUID(uuid),
-            updatedLoadMetaDataDetails)
+          CarbonTablePath.getTableStatusFilePathWithUUID(carbonTable.getTablePath, uuid),
+          updatedLoadMetaDataDetails)
         carbonLoadModel.setLoadMetadataDetails(updatedLoadMetaDataDetails.toList.asJava)
       } finally {
         // check if any other segments needs compaction on in case of MINOR_COMPACTION.
@@ -105,11 +105,9 @@ class AggregateDataMapCompactor(carbonLoadModel: CarbonLoadModel,
         //  4. Therefore tablestatus file will be committed in between multiple commits.
         if (!compactionModel.compactionType.equals(CompactionType.MAJOR)) {
           if (!identifySegmentsToBeMerged().isEmpty) {
-            val carbonTablePath = CarbonStorePath
-              .getCarbonTablePath(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-                .getAbsoluteTableIdentifier)
-            val uuidTableStaus = carbonTablePath.getTableStatusFilePathWithUUID(uuid)
-            val tableStatus = carbonTablePath.getTableStatusFilePath
+            val uuidTableStaus = CarbonTablePath.getTableStatusFilePathWithUUID(
+              carbonTable.getTablePath, uuid)
+            val tableStatus = CarbonTablePath.getTableStatusFilePath(carbonTable.getTablePath)
             if (!uuidTableStaus.equalsIgnoreCase(tableStatus)) {
               FileFactory.getCarbonFile(uuidTableStaus).renameForce(tableStatus)
             }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
index 657e0c5..083b8f2 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -290,13 +290,12 @@ object CommitPreAggregateListener extends OperationEventListener {
       // keep committing until one fails
       val renamedDataMaps = childLoadCommands.takeWhile { childLoadCommand =>
         val childCarbonTable = childLoadCommand.table
-        val carbonTablePath =
-          new CarbonTablePath(childCarbonTable.getCarbonTableIdentifier,
-            childCarbonTable.getTablePath)
         // Generate table status file name with UUID, forExample: tablestatus_1
-        val oldTableSchemaPath = carbonTablePath.getTableStatusFilePathWithUUID(uuid)
+        val oldTableSchemaPath = CarbonTablePath.getTableStatusFilePathWithUUID(
+          childCarbonTable.getTablePath, uuid)
         // Generate table status file name without UUID, forExample: tablestatus
-        val newTableSchemaPath = carbonTablePath.getTableStatusFilePath
+        val newTableSchemaPath = CarbonTablePath.getTableStatusFilePath(
+          childCarbonTable.getTablePath)
         renameDataMapTableStatusFiles(oldTableSchemaPath, newTableSchemaPath, uuid)
       }
       // if true then the commit for one of the child tables has failed
@@ -306,11 +305,11 @@ object CommitPreAggregateListener extends OperationEventListener {
         renamedDataMaps.foreach {
           loadCommand =>
             val carbonTable = loadCommand.table
-            val carbonTablePath = new CarbonTablePath(carbonTable.getCarbonTableIdentifier,
-              carbonTable.getTablePath)
             // rename the backup tablestatus i.e tablestatus_backup_UUID to tablestatus
-            val backupTableSchemaPath = carbonTablePath.getTableStatusFilePath + "_backup_" + uuid
-            val tableSchemaPath = carbonTablePath.getTableStatusFilePath
+            val backupTableSchemaPath = CarbonTablePath.getTableStatusFilePath(
+              carbonTable.getTablePath) + "_backup_" + uuid
+            val tableSchemaPath = CarbonTablePath.getTableStatusFilePath(
+              carbonTable.getTablePath)
             markInProgressSegmentAsDeleted(backupTableSchemaPath, operationContext, loadCommand)
             renameDataMapTableStatusFiles(backupTableSchemaPath, tableSchemaPath, "")
         }
@@ -377,9 +376,8 @@ object CommitPreAggregateListener extends OperationEventListener {
       operationContext: OperationContext,
       uuid: String): Unit = {
     childTables.foreach { childTable =>
-      val carbonTablePath = new CarbonTablePath(childTable.getCarbonTableIdentifier,
-        childTable.getTablePath)
-      val metaDataDir = FileFactory.getCarbonFile(carbonTablePath.getMetadataDirectoryPath)
+      val metaDataDir = FileFactory.getCarbonFile(
+        CarbonTablePath.getMetadataPath(childTable.getTablePath))
       val tableStatusFiles = metaDataDir.listFiles(new CarbonFileFilter {
         override def accept(file: CarbonFile): Boolean = {
           file.getName.contains(uuid) || file.getName.contains("backup")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
index f4077e6..91d1c1b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDataTypeChangeCommand.scala
@@ -26,11 +26,10 @@ import org.apache.spark.util.AlterTableUtil
 
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
-import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 import org.apache.carbondata.events.{AlterTableDataTypeChangePostEvent, AlterTableDataTypeChangePreEvent, OperationContext, OperationListenerBus}
-import org.apache.carbondata.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
+import org.apache.carbondata.format.SchemaEvolutionEntry
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
 
 private[sql] case class CarbonAlterTableDataTypeChangeCommand(
@@ -75,16 +74,18 @@ private[sql] case class CarbonAlterTableDataTypeChangeCommand(
         throwMetadataException(dbName, tableName, s"Invalid Column: $columnName")
       }
       // read the latest schema file
-      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTable)(sparkSession)
+      val tableInfo: org.apache.carbondata.format.TableInfo =
+        metastore.getThriftTableInfo(carbonTable)(sparkSession)
       // maintain the added column for schema evolution history
-      var addColumnSchema: ColumnSchema = null
-      var deletedColumnSchema: ColumnSchema = null
+      var addColumnSchema: org.apache.carbondata.format.ColumnSchema = null
+      var deletedColumnSchema: org.apache.carbondata.format.ColumnSchema = null
       val columnSchemaList = tableInfo.fact_table.table_columns.asScala.filter(!_.isInvisible)
       columnSchemaList.foreach { columnSchema =>
         if (columnSchema.column_name.equalsIgnoreCase(columnName)) {
           deletedColumnSchema = columnSchema.deepCopy
-          columnSchema.setData_type(DataTypeConverterUtil
-            .convertToThriftDataType(alterTableDataTypeChangeModel.dataTypeInfo.dataType))
+          columnSchema.setData_type(
+            DataTypeConverterUtil.convertToThriftDataType(
+              alterTableDataTypeChangeModel.dataTypeInfo.dataType))
           columnSchema.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
           columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
           addColumnSchema = columnSchema

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
index a55dbdd..870c140 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala
@@ -138,7 +138,8 @@ private[sql] case class CarbonAlterTableRenameCommand(
           sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
         }
       }
-      newTablePath = metastore.updateTableSchemaForAlter(newTableIdentifier,
+      newTablePath = metastore.updateTableSchemaForAlter(
+        newTableIdentifier,
         carbonTable.getCarbonTableIdentifier,
         tableInfo,
         schemaEvolutionEntry,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
index 0645040..7c40bcd 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonMetaStore.scala
@@ -27,8 +27,7 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTable
 import org.apache.carbondata.core.metadata.schema
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
+import org.apache.carbondata.format.SchemaEvolutionEntry
 
 
 /**
@@ -69,7 +68,8 @@ trait CarbonMetaStore {
    * @param carbonStorePath
    * @param sparkSession
    */
-  def updateTableSchemaForAlter(newTableIdentifier: CarbonTableIdentifier,
+  def updateTableSchemaForAlter(
+      newTableIdentifier: CarbonTableIdentifier,
       oldTableIdentifier: CarbonTableIdentifier,
       thriftTableInfo: org.apache.carbondata.format.TableInfo,
       schemaEvolutionEntry: SchemaEvolutionEntry,
@@ -128,7 +128,8 @@ trait CarbonMetaStore {
   def removeTableFromMetadata(dbName: String, tableName: String): Unit
 
   def updateMetadataByThriftTable(schemaFilePath: String,
-      tableInfo: TableInfo, dbName: String, tableName: String, tablePath: String): Unit
+      tableInfo: org.apache.carbondata.format.TableInfo,
+      dbName: String, tableName: String, tablePath: String): Unit
 
   def isTablePathExists(tableIdentifier: TableIdentifier)(sparkSession: SparkSession): Boolean
 
@@ -143,7 +144,8 @@ trait CarbonMetaStore {
 
   def listAllTables(sparkSession: SparkSession): Seq[CarbonTable]
 
-  def getThriftTableInfo(carbonTable: CarbonTable)(sparkSession: SparkSession): TableInfo
+  def getThriftTableInfo(carbonTable: CarbonTable)(sparkSession: SparkSession):
+  org.apache.carbondata.format.TableInfo
 
   def getTableFromMetadataCache(database: String, tableName: String): Option[CarbonTable]
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/806b9841/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index e7c52f6..03c8c27 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -180,9 +180,10 @@ public final class CarbonLoaderUtil {
     }
     String tableStatusPath;
     if (loadModel.getCarbonDataLoadSchema().getCarbonTable().isChildDataMap() && !uuid.isEmpty()) {
-      tableStatusPath = CarbonTablePath.getTableStatusFilePathWithUUID(uuid);
+      tableStatusPath = CarbonTablePath.getTableStatusFilePathWithUUID(
+          identifier.getTablePath(), uuid);
     } else {
-      tableStatusPath = CarbonTablePath.getTableStatusFilePath();
+      tableStatusPath = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
     }
     SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
     ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();


[40/49] carbondata git commit: [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

Posted by ja...@apache.org.
[CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

To make assembling JAR of store-sdk module, it should not depend on carbon-spark module

This closes #1970


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

Branch: refs/heads/carbonstore-rebase4
Commit: 06a715860116f390b4cb605c5aaec74cead85d75
Parents: 599b9ae
Author: Jacky Li <ja...@qq.com>
Authored: Sun Feb 11 21:37:04 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:20:21 2018 +0800

----------------------------------------------------------------------
 .../java/org/apache/carbondata/common/Maps.java |  39 ++
 .../org/apache/carbondata/common/Strings.java   |   3 +
 .../ConcurrentOperationException.java           |  50 ++
 .../exceptions/TableStatusLockException.java    |  34 ++
 .../sql/InvalidLoadOptionException.java         |  33 +
 .../sql/MalformedCarbonCommandException.java    |  75 +++
 .../sql/MalformedDataMapCommandException.java   |  37 ++
 .../exceptions/sql/NoSuchDataMapException.java  |  39 ++
 .../statusmanager/SegmentStatusManager.java     | 124 ++++
 .../carbondata/core/util/DeleteLoadFolders.java | 210 +++++++
 .../preaggregate/TestPreAggCreateCommand.scala  |   2 +-
 .../preaggregate/TestPreAggregateDrop.scala     |   2 +-
 .../timeseries/TestTimeSeriesCreateTable.scala  |   2 +-
 .../timeseries/TestTimeSeriesDropSuite.scala    |   2 +-
 .../TestTimeseriesTableSelection.scala          |   2 +-
 .../TestDataLoadWithColumnsMoreThanSchema.scala |   3 +-
 .../dataload/TestGlobalSortDataLoad.scala       |   2 +-
 .../TestLoadDataWithDiffTimestampFormat.scala   |   2 +-
 .../TestLoadDataWithFileHeaderException.scala   |  11 +-
 ...ataWithMalformedCarbonCommandException.scala |   3 +-
 .../testsuite/dataload/TestLoadOptions.scala    |   2 +-
 .../dataload/TestTableLevelBlockSize.scala      |   4 +-
 .../testsuite/datamap/TestDataMapCommand.scala  |   2 +-
 .../dataretention/DataRetentionTestCase.scala   |   2 +-
 .../spark/testsuite/datetype/DateTypeTest.scala |   2 +-
 .../testsuite/sortcolumns/TestSortColumns.scala |   3 +-
 integration/spark-common/pom.xml                |   5 -
 .../exception/ConcurrentOperationException.java |  38 --
 .../MalformedCarbonCommandException.java        |  69 ---
 .../MalformedDataMapCommandException.java       |  32 -
 .../spark/exception/NoSuchDataMapException.java |  33 -
 .../org/apache/carbondata/api/CarbonStore.scala |   3 +-
 .../spark/CarbonColumnValidator.scala           |   8 +-
 .../carbondata/spark/load/ValidateUtil.scala    |  72 ---
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   6 +-
 .../carbondata/spark/util/CommonUtil.scala      |  70 +--
 .../carbondata/spark/util/DataLoadingUtil.scala | 610 -------------------
 .../spark/util/GlobalDictionaryUtil.scala       |   2 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   4 +-
 .../spark/rdd/CarbonTableCompactor.scala        |   2 +-
 .../org/apache/spark/sql/CarbonSource.scala     |   2 +-
 .../datamap/CarbonCreateDataMapCommand.scala    |   2 +-
 .../datamap/CarbonDropDataMapCommand.scala      |   2 +-
 .../CarbonAlterTableCompactionCommand.scala     |  24 +-
 .../management/CarbonLoadDataCommand.scala      |  17 +-
 .../CarbonProjectForDeleteCommand.scala         |   2 +-
 .../CarbonProjectForUpdateCommand.scala         |   2 +-
 .../command/mutation/IUDCommonUtil.scala        |   2 +-
 .../CreatePreAggregateTableCommand.scala        |   7 +-
 .../preaaggregate/PreAggregateUtil.scala        |   2 +-
 .../schema/CarbonAlterTableRenameCommand.scala  |   3 +-
 .../command/timeseries/TimeSeriesUtil.scala     |   2 +-
 .../datasources/CarbonFileFormat.scala          |  14 +-
 .../sql/execution/strategy/DDLStrategy.scala    |   2 +-
 .../strategy/StreamingTableStrategy.scala       |   2 +-
 .../execution/command/CarbonHiveCommands.scala  |   2 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   2 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala |   2 +-
 .../org/apache/spark/util/AlterTableUtil.scala  |   2 +-
 .../org/apache/spark/util/TableAPIUtil.scala    |   2 +-
 .../spark/sql/hive/CarbonSessionState.scala     |   3 +-
 .../segmentreading/TestSegmentReading.scala     |   2 +-
 .../spark/util/AllDictionaryTestCase.scala      |   4 +-
 .../util/ExternalColumnDictionaryTestCase.scala |   6 +-
 .../TestStreamingTableOperation.scala           |   4 +-
 .../bucketing/TableBucketingTestCase.scala      |   2 +-
 .../vectorreader/AddColumnTestCases.scala       |   2 +-
 .../loading/model/CarbonLoadModel.java          |  14 +-
 .../loading/model/CarbonLoadModelBuilder.java   | 322 ++++++++++
 .../processing/loading/model/LoadOption.java    | 251 ++++++++
 .../processing/util/CarbonLoaderUtil.java       |  83 +--
 .../processing/util/DeleteLoadFolders.java      | 210 -------
 store/sdk/pom.xml                               |   2 +-
 .../sdk/file/CarbonWriterBuilder.java           |  15 +-
 .../sdk/file/CSVCarbonWriterSuite.java          |   2 +-
 .../streaming/StreamSinkFactory.scala           |  11 +-
 77 files changed, 1352 insertions(+), 1322 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/common/src/main/java/org/apache/carbondata/common/Maps.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/Maps.java b/common/src/main/java/org/apache/carbondata/common/Maps.java
new file mode 100644
index 0000000..14fc329
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/Maps.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common;
+
+import java.util.Map;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+
+@InterfaceAudience.Developer
+public class Maps {
+
+  /**
+   * Return value if key is contained in the map, else return defauleValue.
+   * This is added to avoid JDK 8 dependency
+   */
+  public static <K, V> V getOrDefault(Map<K, V> map, K key, V defaultValue) {
+    V value = map.get(key);
+    if (value != null) {
+      return value;
+    } else {
+      return defaultValue;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/common/src/main/java/org/apache/carbondata/common/Strings.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/Strings.java b/common/src/main/java/org/apache/carbondata/common/Strings.java
index 23288dd..08fdc3c 100644
--- a/common/src/main/java/org/apache/carbondata/common/Strings.java
+++ b/common/src/main/java/org/apache/carbondata/common/Strings.java
@@ -19,6 +19,9 @@ package org.apache.carbondata.common;
 
 import java.util.Objects;
 
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+
+@InterfaceAudience.Developer
 public class Strings {
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/common/src/main/java/org/apache/carbondata/common/exceptions/ConcurrentOperationException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/ConcurrentOperationException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/ConcurrentOperationException.java
new file mode 100644
index 0000000..7308100
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/ConcurrentOperationException.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.exceptions;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+/**
+ * This exception will be thrown when executing concurrent operations which
+ * is not supported in carbon.
+ *
+ * For example, when INSERT OVERWRITE is executing, other operations are not
+ * allowed, so this exception will be thrown
+ */
+@InterfaceAudience.User
+@InterfaceStability.Stable
+public class ConcurrentOperationException extends Exception {
+
+  public ConcurrentOperationException(String dbName, String tableName, String command1,
+      String command2) {
+    super(command1 + " is in progress for table " + dbName + "." + tableName + ", " + command2 +
+      " operation is not allowed");
+  }
+
+  public ConcurrentOperationException(CarbonTable table, String command1, String command2) {
+    this(table.getDatabaseName(), table.getTableName(), command1, command2);
+  }
+
+  public String getMessage() {
+    return super.getMessage();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/common/src/main/java/org/apache/carbondata/common/exceptions/TableStatusLockException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/TableStatusLockException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/TableStatusLockException.java
new file mode 100644
index 0000000..89cfd46
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/TableStatusLockException.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.exceptions;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * This exception will be thrown when failed to acquire lock for table status metadata,
+ * or re-try timed out
+ */
+@InterfaceAudience.User
+@InterfaceStability.Evolving
+public class TableStatusLockException extends RuntimeException {
+
+  public TableStatusLockException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/common/src/main/java/org/apache/carbondata/common/exceptions/sql/InvalidLoadOptionException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/InvalidLoadOptionException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/InvalidLoadOptionException.java
new file mode 100644
index 0000000..41b2434
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/InvalidLoadOptionException.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.exceptions.sql;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * This exception will be thrown when loading option is invalid for SQL
+ * loading statement (LOAD DATA, INSERT INTO)
+ */
+@InterfaceAudience.User
+@InterfaceStability.Stable
+public class InvalidLoadOptionException extends MalformedCarbonCommandException {
+  public InvalidLoadOptionException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedCarbonCommandException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedCarbonCommandException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedCarbonCommandException.java
new file mode 100644
index 0000000..5fe3ce8
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedCarbonCommandException.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.exceptions.sql;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * This exception will be thrown if any validation fails then parsing
+ * SQL statement.
+ */
+@InterfaceAudience.User
+@InterfaceStability.Stable
+public class MalformedCarbonCommandException extends Exception {
+
+  /**
+   * default serial version ID.
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * The Error message.
+   */
+  private String msg = "";
+
+  /**
+   * Constructor
+   *
+   * @param msg The error message for this exception.
+   */
+  public MalformedCarbonCommandException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param msg The error message for this exception.
+   */
+  public MalformedCarbonCommandException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * getLocalizedMessage
+   */
+  @Override
+  public String getLocalizedMessage() {
+    return super.getLocalizedMessage();
+  }
+
+  /**
+   * getMessage
+   */
+  public String getMessage() {
+    return this.msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
new file mode 100644
index 0000000..7c25b2c
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedDataMapCommandException.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.exceptions.sql;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * This exception will be thrown when Datamap related SQL statement is invalid
+ */
+@InterfaceAudience.User
+@InterfaceStability.Stable
+public class MalformedDataMapCommandException extends MalformedCarbonCommandException {
+  /**
+   * default serial version ID.
+   */
+  private static final long serialVersionUID = 1L;
+
+  public MalformedDataMapCommandException(String msg) {
+    super(msg);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchDataMapException.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchDataMapException.java b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchDataMapException.java
new file mode 100644
index 0000000..7ab9048
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchDataMapException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common.exceptions.sql;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * This exception will be thrown if datamap is not found when executing datamap
+ * related SQL statement
+ */
+@InterfaceAudience.User
+@InterfaceStability.Stable
+public class NoSuchDataMapException extends MalformedCarbonCommandException {
+
+  /**
+   * default serial version ID.
+   */
+  private static final long serialVersionUID = 1L;
+
+  public NoSuchDataMapException(String dataMapName, String tableName) {
+    super("Datamap with name " + dataMapName + " does not exist under table " + tableName);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
index 1b45be7..89666ab 100755
--- a/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
@@ -30,6 +30,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.carbondata.common.exceptions.TableStatusLockException;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
@@ -47,6 +48,7 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.DeleteLoadFolders;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import com.google.gson.Gson;
@@ -765,4 +767,126 @@ public class SegmentStatusManager {
     }
   }
 
+  private static boolean isLoadDeletionRequired(String metaDataLocation) {
+    LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metaDataLocation);
+    if (details != null && details.length > 0) {
+      for (LoadMetadataDetails oneRow : details) {
+        if ((SegmentStatus.MARKED_FOR_DELETE == oneRow.getSegmentStatus()
+            || SegmentStatus.COMPACTED == oneRow.getSegmentStatus()
+            || SegmentStatus.INSERT_IN_PROGRESS == oneRow.getSegmentStatus()
+            || SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneRow.getSegmentStatus())
+            && oneRow.getVisibility().equalsIgnoreCase("true")) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * This will update the old table status details before clean files to the latest table status.
+   * @param oldList
+   * @param newList
+   * @return
+   */
+  public static List<LoadMetadataDetails> updateLoadMetadataFromOldToNew(
+      LoadMetadataDetails[] oldList, LoadMetadataDetails[] newList) {
+
+    List<LoadMetadataDetails> newListMetadata =
+        new ArrayList<LoadMetadataDetails>(Arrays.asList(newList));
+    for (LoadMetadataDetails oldSegment : oldList) {
+      if ("false".equalsIgnoreCase(oldSegment.getVisibility())) {
+        newListMetadata.get(newListMetadata.indexOf(oldSegment)).setVisibility("false");
+      }
+    }
+    return newListMetadata;
+  }
+
+  private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
+      List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
+    String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
+
+    DataOutputStream dataOutputStream;
+    Gson gsonObjectToWrite = new Gson();
+    BufferedWriter brWriter = null;
+
+    AtomicFileOperations writeOperation =
+        new AtomicFileOperationsImpl(dataLoadLocation, FileFactory.getFileType(dataLoadLocation));
+
+    try {
+
+      dataOutputStream = writeOperation.openForWrite(FileWriteOperation.OVERWRITE);
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      String metadataInstance = gsonObjectToWrite.toJson(listOfLoadFolderDetails.toArray());
+      brWriter.write(metadataInstance);
+    } finally {
+      try {
+        if (null != brWriter) {
+          brWriter.flush();
+        }
+      } catch (Exception e) {
+        LOG.error("error in  flushing ");
+
+      }
+      CarbonUtil.closeStreams(brWriter);
+      writeOperation.close();
+    }
+  }
+
+  public static void deleteLoadsAndUpdateMetadata(
+      CarbonTable carbonTable,
+      boolean isForceDeletion) throws IOException {
+    if (isLoadDeletionRequired(carbonTable.getMetadataPath())) {
+      LoadMetadataDetails[] details =
+          SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
+      AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
+      ICarbonLock carbonTableStatusLock = CarbonLockFactory.getCarbonLockObj(
+          identifier, LockUsage.TABLE_STATUS_LOCK);
+
+      // Delete marked loads
+      boolean isUpdationRequired = DeleteLoadFolders.deleteLoadFoldersFromFileSystem(
+          identifier, isForceDeletion, details, carbonTable.getMetadataPath());
+
+      boolean updationCompletionStatus = false;
+
+      if (isUpdationRequired) {
+        try {
+          // Update load metadate file after cleaning deleted nodes
+          if (carbonTableStatusLock.lockWithRetries()) {
+            LOG.info("Table status lock has been successfully acquired.");
+
+            // read latest table status again.
+            LoadMetadataDetails[] latestMetadata =
+                SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath());
+
+            // update the metadata details from old to new status.
+            List<LoadMetadataDetails> latestStatus =
+                updateLoadMetadataFromOldToNew(details, latestMetadata);
+
+            writeLoadMetadata(identifier, latestStatus);
+          } else {
+            String dbName = identifier.getCarbonTableIdentifier().getDatabaseName();
+            String tableName = identifier.getCarbonTableIdentifier().getTableName();
+            String errorMsg = "Clean files request is failed for " +
+                dbName + "." + tableName +
+                ". Not able to acquire the table status lock due to other operation " +
+                "running in the background.";
+            LOG.audit(errorMsg);
+            LOG.error(errorMsg);
+            throw new TableStatusLockException(errorMsg + " Please try after some time.");
+          }
+          updationCompletionStatus = true;
+        } finally {
+          CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.TABLE_STATUS_LOCK);
+          if (updationCompletionStatus) {
+            DeleteLoadFolders.physicalFactAndMeasureMetadataDeletion(
+                identifier, carbonTable.getMetadataPath(), isForceDeletion);
+          }
+        }
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java b/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
new file mode 100644
index 0000000..d1cd4fb
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.util;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.indexstore.PartitionSpec;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.SegmentFileStore;
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+public final class DeleteLoadFolders {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DeleteLoadFolders.class.getName());
+
+  private DeleteLoadFolders() {
+
+  }
+
+  /**
+   * returns segment path
+   *
+   * @param identifier
+   * @param oneLoad
+   * @return
+   */
+  private static String getSegmentPath(AbsoluteTableIdentifier identifier,
+      LoadMetadataDetails oneLoad) {
+    String segmentId = oneLoad.getLoadName();
+    return CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
+  }
+
+  public static void physicalFactAndMeasureMetadataDeletion(
+      AbsoluteTableIdentifier absoluteTableIdentifier, String metadataPath, boolean isForceDelete,
+      List<PartitionSpec> specs) {
+    LoadMetadataDetails[] currentDetails = SegmentStatusManager.readLoadMetadata(metadataPath);
+    for (LoadMetadataDetails oneLoad : currentDetails) {
+      if (checkIfLoadCanBeDeletedPhysically(oneLoad, isForceDelete)) {
+        try {
+          if (oneLoad.getSegmentFile() != null) {
+            SegmentFileStore
+                .deleteSegment(absoluteTableIdentifier.getTablePath(), oneLoad.getSegmentFile(),
+                    specs);
+          } else {
+            String path = getSegmentPath(absoluteTableIdentifier, oneLoad);
+            boolean status = false;
+            if (FileFactory.isFileExist(path, FileFactory.getFileType(path))) {
+              CarbonFile file = FileFactory.getCarbonFile(path, FileFactory.getFileType(path));
+              CarbonFile[] filesToBeDeleted = file.listFiles(new CarbonFileFilter() {
+
+                @Override public boolean accept(CarbonFile file) {
+                  return (CarbonTablePath.isCarbonDataFile(file.getName()) ||
+                      CarbonTablePath.isCarbonIndexFile(file.getName()));
+                }
+              });
+
+              //if there are no fact and msr metadata files present then no need to keep
+              //entry in metadata.
+              if (filesToBeDeleted.length == 0) {
+                status = true;
+              } else {
+
+                for (CarbonFile eachFile : filesToBeDeleted) {
+                  if (!eachFile.delete()) {
+                    LOGGER.warn("Unable to delete the file as per delete command " + eachFile
+                        .getAbsolutePath());
+                    status = false;
+                  } else {
+                    status = true;
+                  }
+                }
+              }
+              // need to delete the complete folder.
+              if (status) {
+                if (!file.delete()) {
+                  LOGGER.warn("Unable to delete the folder as per delete command " + file
+                      .getAbsolutePath());
+                }
+              }
+
+            } else {
+              LOGGER.warn("Files are not found in segment " + path
+                  + " it seems, files are already being deleted");
+            }
+
+          }
+        } catch (IOException e) {
+          LOGGER.warn("Unable to delete the file as per delete command " + oneLoad.getLoadName());
+        }
+      }
+    }
+  }
+
+  private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
+      boolean isForceDelete) {
+    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
+        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
+        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
+        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
+        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      if (isForceDelete) {
+        return true;
+      }
+      long deletionTime = oneLoad.getModificationOrdeletionTimesStamp();
+
+      return CarbonUpdateUtil.isMaxQueryTimeoutExceeded(deletionTime);
+
+    }
+
+    return false;
+  }
+
+  private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
+      boolean isForceDelete) {
+    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
+        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus())) {
+      if (isForceDelete) {
+        return true;
+      }
+      long deletionTime = oneLoad.getModificationOrdeletionTimesStamp();
+
+      return CarbonUpdateUtil.isMaxQueryTimeoutExceeded(deletionTime);
+
+    }
+
+    return false;
+  }
+
+  private static LoadMetadataDetails getCurrentLoadStatusOfSegment(String segmentId,
+      String metadataPath) {
+    LoadMetadataDetails[] currentDetails = SegmentStatusManager.readLoadMetadata(metadataPath);
+    for (LoadMetadataDetails oneLoad : currentDetails) {
+      if (oneLoad.getLoadName().equalsIgnoreCase(segmentId)) {
+        return oneLoad;
+      }
+    }
+    return null;
+  }
+
+  public static boolean deleteLoadFoldersFromFileSystem(
+      AbsoluteTableIdentifier absoluteTableIdentifier, boolean isForceDelete,
+      LoadMetadataDetails[] details, String metadataPath) {
+    boolean isDeleted = false;
+    if (details != null && details.length != 0) {
+      for (LoadMetadataDetails oneLoad : details) {
+        if (checkIfLoadCanBeDeleted(oneLoad, isForceDelete)) {
+          ICarbonLock segmentLock = CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier,
+              CarbonTablePath.addSegmentPrefix(oneLoad.getLoadName()) + LockUsage.LOCK);
+          try {
+            if (oneLoad.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS
+                || oneLoad.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS) {
+              if (segmentLock.lockWithRetries(1, 5)) {
+                LOGGER.info("Info: Acquired segment lock on segment:" + oneLoad.getLoadName());
+                LoadMetadataDetails currentDetails =
+                    getCurrentLoadStatusOfSegment(oneLoad.getLoadName(), metadataPath);
+                if (currentDetails != null && checkIfLoadCanBeDeleted(currentDetails,
+                    isForceDelete)) {
+                  oneLoad.setVisibility("false");
+                  isDeleted = true;
+                  LOGGER.info("Info: Deleted the load " + oneLoad.getLoadName());
+                }
+              } else {
+                LOGGER.info("Info: Load in progress for segment" + oneLoad.getLoadName());
+                return isDeleted;
+              }
+            } else {
+              oneLoad.setVisibility("false");
+              isDeleted = true;
+              LOGGER.info("Info: Deleted the load " + oneLoad.getLoadName());
+            }
+          } finally {
+            segmentLock.unlock();
+            LOGGER.info("Info: Segment lock on segment:" + oneLoad.getLoadName() + " is released");
+          }
+        }
+      }
+    }
+    return isDeleted;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
index 1e59a80..8f63af6 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
@@ -26,10 +26,10 @@ import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.metadata.encoder.Encoding
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES
-import org.apache.carbondata.spark.exception.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 
 class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
index 911a725..2c7c593 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggregateDrop.scala
@@ -20,7 +20,7 @@ package org.apache.carbondata.integration.spark.testsuite.preaggregate
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.spark.exception.NoSuchDataMapException
+import org.apache.carbondata.common.exceptions.sql.NoSuchDataMapException
 
 class TestPreAggregateDrop extends QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
index 3d991a9..43316b3 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesCreateTable.scala
@@ -20,8 +20,8 @@ import org.apache.spark.sql.AnalysisException
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedDataMapCommandException}
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES
-import org.apache.carbondata.spark.exception.{MalformedDataMapCommandException, MalformedCarbonCommandException}
 
 class TestTimeSeriesCreateTable extends QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesDropSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesDropSuite.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesDropSuite.scala
index 545c4de..5fe21e8 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesDropSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeSeriesDropSuite.scala
@@ -19,7 +19,7 @@ package org.apache.carbondata.integration.spark.testsuite.timeseries
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 
 class TestTimeSeriesDropSuite extends QueryTest with BeforeAndAfterAll with BeforeAndAfterEach {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesTableSelection.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesTableSelection.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesTableSelection.scala
index 3065952..3f140df 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesTableSelection.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/timeseries/TestTimeseriesTableSelection.scala
@@ -24,8 +24,8 @@ import org.apache.spark.sql.test.util.QueryTest
 import org.apache.spark.util.SparkUtil4Test
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.metadata.schema.datamap.DataMapProvider.TIMESERIES
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 class TestTimeseriesTableSelection extends QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
index 1532328..4e5ebbb 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
@@ -18,9 +18,10 @@
 package org.apache.carbondata.spark.testsuite.dataload
 
 import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.spark.sql.test.util.QueryTest
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+
 /**
  * This class will test data load in which number of columns in data are more than
  * the number of columns in schema

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
index cbbb191..3babf4f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestGlobalSortDataLoad.scala
@@ -23,13 +23,13 @@ import org.apache.commons.io.FileUtils
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.execution.BatchedDataSourceScanExec
 import org.apache.spark.sql.test.TestQueryExecutor.projectPath
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.indexstore.blockletindex.SegmentIndexFileStore
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.spark.rdd.CarbonScanRDD

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
index ec6fff1..c06d782 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithDiffTimestampFormat.scala
@@ -25,10 +25,10 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.spark.sql.test.util.QueryTest
 
 import org.apache.carbondata.common.constants.LoggerAction
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 
 class TestLoadDataWithDiffTimestampFormat extends QueryTest with BeforeAndAfterAll {
   val bad_records_action = CarbonProperties.getInstance()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithFileHeaderException.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithFileHeaderException.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithFileHeaderException.scala
index 7700ed5..edcdd51 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithFileHeaderException.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithFileHeaderException.scala
@@ -20,6 +20,9 @@ package org.apache.carbondata.spark.testsuite.dataload
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
+
 class TestLoadDataWithFileHeaderException extends QueryTest with BeforeAndAfterAll{
   override def beforeAll {
     sql("DROP TABLE IF EXISTS t3")
@@ -32,7 +35,7 @@ class TestLoadDataWithFileHeaderException extends QueryTest with BeforeAndAfterA
   }
 
   test("test load data both file and ddl without file header exception") {
-    val e = intercept[Exception] {
+    val e = intercept[CarbonDataLoadingException] {
       sql(
         s"""LOAD DATA LOCAL INPATH '$resourcesPath/source_without_header.csv' into table t3""")
     }
@@ -41,7 +44,7 @@ class TestLoadDataWithFileHeaderException extends QueryTest with BeforeAndAfterA
   }
 
   test("test load data ddl provided wrong file header exception") {
-    val e = intercept[Exception] {
+    val e = intercept[CarbonDataLoadingException] {
       sql(
         s"""
            LOAD DATA LOCAL INPATH '$resourcesPath/source_without_header.csv' into table t3
@@ -52,7 +55,7 @@ class TestLoadDataWithFileHeaderException extends QueryTest with BeforeAndAfterA
   }
 
   test("test load data with wrong header , but without fileheader") {
-    val e = intercept[Exception] {
+    val e = intercept[InvalidLoadOptionException] {
       sql(
         s"""
            LOAD DATA LOCAL INPATH '$resourcesPath/source.csv' into table t3
@@ -63,7 +66,7 @@ class TestLoadDataWithFileHeaderException extends QueryTest with BeforeAndAfterA
   }
 
   test("test load data with wrong header and fileheader") {
-    val e = intercept[Exception] {
+    val e = intercept[InvalidLoadOptionException] {
       sql(
         s"""
          LOAD DATA LOCAL INPATH '$resourcesPath/source_without_header.csv' into table t3

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
index 1851705..6759049 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
@@ -18,9 +18,10 @@
 package org.apache.carbondata.spark.testsuite.dataload
 
 import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.spark.sql.test.util.QueryTest
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+
 class TestLoadDataWithMalformedCarbonCommandException extends QueryTest with BeforeAndAfterAll {
 
   override def beforeAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadOptions.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadOptions.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadOptions.scala
index d2c7e63..4ec9335 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadOptions.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadOptions.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.Row
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 
 class TestLoadOptions extends QueryTest with BeforeAndAfterAll{
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestTableLevelBlockSize.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestTableLevelBlockSize.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestTableLevelBlockSize.scala
index a77b210..f6a049a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestTableLevelBlockSize.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestTableLevelBlockSize.scala
@@ -19,11 +19,13 @@ package org.apache.carbondata.spark.testsuite.dataload
 
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.spark.sql.test.util.QueryTest
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+
 /**
   * Test Class for table block size
   *

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
index d60b7db..6ac9c7a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/TestDataMapCommand.scala
@@ -23,11 +23,11 @@ import org.apache.spark.sql.Row
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedDataMapCommandException
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.CarbonMetadata
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
-import org.apache.carbondata.spark.exception.MalformedDataMapCommandException
 
 class TestDataMapCommand extends QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
index 7c82f75..a70584b 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
@@ -29,9 +29,9 @@ import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.spark.sql.test.util.QueryTest
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datetype/DateTypeTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datetype/DateTypeTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datetype/DateTypeTest.scala
index e2df07c..b9b01f8 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datetype/DateTypeTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datetype/DateTypeTest.scala
@@ -16,10 +16,10 @@
  */
 package org.apache.carbondata.spark.testsuite.datetype
 
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.spark.sql.test.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 
 class DateTypeTest extends QueryTest with BeforeAndAfterAll{
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
index 51df525..17beace 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/sortcolumns/TestSortColumns.scala
@@ -21,10 +21,11 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.spark.sql.test.util.QueryTest
 import org.apache.spark.util.SparkUtil4Test
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+
 class TestSortColumns extends QueryTest with BeforeAndAfterAll {
 
   override def beforeAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common/pom.xml b/integration/spark-common/pom.xml
index 005117f..295d62b 100644
--- a/integration/spark-common/pom.xml
+++ b/integration/spark-common/pom.xml
@@ -36,11 +36,6 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
-      <artifactId>carbondata-processing</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.carbondata</groupId>
       <artifactId>carbondata-hadoop</artifactId>
       <version>${project.version}</version>
     </dependency>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ConcurrentOperationException.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ConcurrentOperationException.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ConcurrentOperationException.java
deleted file mode 100644
index cc0047f..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/ConcurrentOperationException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.exception;
-
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-
-public class ConcurrentOperationException extends MalformedCarbonCommandException {
-
-  public ConcurrentOperationException(String dbName, String tableName, String command1,
-      String command2) {
-    super(command1 + " is in progress for table " + dbName + "." + tableName + ", " + command2 +
-      " operation is not allowed");
-  }
-
-  public ConcurrentOperationException(CarbonTable table, String command1, String command2) {
-    this(table.getDatabaseName(), table.getTableName(), command1, command2);
-  }
-
-  public String getMessage() {
-    return super.getMessage();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/MalformedCarbonCommandException.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/MalformedCarbonCommandException.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/MalformedCarbonCommandException.java
deleted file mode 100644
index 9f441d3..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/MalformedCarbonCommandException.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.exception;
-
-// After parsing carbon query successfully , if any validation fails then
-// use MalformedCarbonCommandException
-public class MalformedCarbonCommandException extends Exception {
-
-
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The Error message.
-   */
-  private String msg = "";
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public MalformedCarbonCommandException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public MalformedCarbonCommandException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override
-  public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/MalformedDataMapCommandException.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/MalformedDataMapCommandException.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/MalformedDataMapCommandException.java
deleted file mode 100644
index a05d8e6..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/MalformedDataMapCommandException.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.exception;
-
-/**
- * Throw exception when using illegal argument
- */
-public class MalformedDataMapCommandException extends MalformedCarbonCommandException {
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  public MalformedDataMapCommandException(String msg) {
-    super(msg);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/NoSuchDataMapException.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/NoSuchDataMapException.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/NoSuchDataMapException.java
deleted file mode 100644
index 959e70d..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/exception/NoSuchDataMapException.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.exception;
-
-/**
- * if the dataMap does not exist, carbon should throw NoSuchDataMapException
- */
-public class NoSuchDataMapException extends MalformedCarbonCommandException {
-
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  public NoSuchDataMapException(String dataMapName, String tableName) {
-    super("Datamap with name " + dataMapName + " does not exist under table " + tableName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
index 8adcb11..b69ec37 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
@@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.unsafe.types.UTF8String
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -35,7 +36,6 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, SegmentFile
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.util.DataLoadingUtil
 
 object CarbonStore {
@@ -139,6 +139,7 @@ object CarbonStore {
         carbonCleanFilesLock =
           CarbonLockUtil
             .getLockObject(absoluteTableIdentifier, LockUsage.CLEAN_FILES_LOCK, errorMsg)
+        SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable, true)
         DataLoadingUtil.deleteLoadsAndUpdateMetadata(
           isForceDeletion = true, carbonTable, currentTablePartitions.map(_.asJava).orNull)
         CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, true)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala
index ad624ee..578138f 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonColumnValidator.scala
@@ -16,12 +16,12 @@
  */
 package org.apache.carbondata.spark
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
- /**
-  * Carbon column validator
-  */
+/**
+ * Carbon column validator
+ */
 class CarbonColumnValidator extends ColumnValidator {
   def validateColumns(allColumns: Seq[ColumnSchema]) {
     allColumns.foreach { columnSchema =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/ValidateUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/ValidateUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/ValidateUtil.scala
index 2fdd2b1..e69de29 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/ValidateUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/ValidateUtil.scala
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.load
-
-import java.text.SimpleDateFormat
-
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.processing.loading.sort.SortScopeOptions
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-
-object ValidateUtil {
-
-  /**
-   * validates both timestamp and date for illegal values
-   *
-   * @param dateTimeLoadFormat
-   * @param dateTimeLoadOption
-   */
-  def validateDateTimeFormat(dateTimeLoadFormat: String, dateTimeLoadOption: String): Unit = {
-    // allowing empty value to be configured for dateformat option.
-    if (dateTimeLoadFormat != null && dateTimeLoadFormat.trim != "") {
-      try {
-        new SimpleDateFormat(dateTimeLoadFormat)
-      } catch {
-        case _: IllegalArgumentException =>
-          throw new MalformedCarbonCommandException(s"Error: Wrong option: $dateTimeLoadFormat is" +
-                                                    s" provided for option $dateTimeLoadOption")
-      }
-    }
-  }
-
-  def validateSortScope(carbonTable: CarbonTable, sortScope: String): Unit = {
-    if (sortScope != null) {
-      // Don't support use global sort on partitioned table.
-      if (carbonTable.getPartitionInfo(carbonTable.getTableName) != null &&
-          !carbonTable.isHivePartitionTable &&
-          sortScope.equalsIgnoreCase(SortScopeOptions.SortScope.GLOBAL_SORT.toString)) {
-        throw new MalformedCarbonCommandException("Don't support use global sort on partitioned " +
-          "table.")
-      }
-    }
-  }
-
-  def validateGlobalSortPartitions(globalSortPartitions: String): Unit = {
-    if (globalSortPartitions != null) {
-      try {
-        val num = globalSortPartitions.toInt
-        if (num <= 0) {
-          throw new MalformedCarbonCommandException("'GLOBAL_SORT_PARTITIONS' should be greater " +
-            "than 0.")
-        }
-      } catch {
-        case e: NumberFormatException => throw new MalformedCarbonCommandException(e.getMessage)
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 3fbed6f..444c9dc 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -299,11 +299,11 @@ class CarbonMergerRDD[K, V](
       val splits = format.getSplits(job)
 
       // keep on assigning till last one is reached.
-      if (null != splits && splits.size > 0) splitsOfLastSegment =
-        splits.asScala
+      if (null != splits && splits.size > 0) {
+        splitsOfLastSegment = splits.asScala
           .map(_.asInstanceOf[CarbonInputSplit])
           .filter { split => FileFormat.COLUMNAR_V3.equals(split.getFileFormat) }.toList.asJava
-
+      }
       carbonInputSplits ++:= splits.asScala.map(_.asInstanceOf[CarbonInputSplit]).filter{ entry =>
         val blockInfo = new TableBlockInfo(entry.getPath.toString,
           entry.getStart, entry.getSegmentId,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 7815c99..9104a32 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -37,6 +37,7 @@ import org.apache.spark.sql.types.{MetadataBuilder, StringType}
 import org.apache.spark.sql.util.CarbonException
 import org.apache.spark.util.FileUtils
 
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -55,7 +56,7 @@ import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
 
 
 object CommonUtil {
@@ -633,13 +634,6 @@ object CommonUtil {
     parsedPropertyValueString
   }
 
-
-  def readLoadMetadataDetails(model: CarbonLoadModel): Unit = {
-    val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetadataPath
-    val details = SegmentStatusManager.readLoadMetadata(metadataPath)
-    model.setLoadMetadataDetails(new util.ArrayList[LoadMetadataDetails](details.toList.asJava))
-  }
-
   def configureCSVInputFormat(configuration: Configuration,
       carbonLoadModel: CarbonLoadModel): Unit = {
     CSVInputFormat.setCommentCharacter(configuration, carbonLoadModel.getCommentChar)
@@ -681,65 +675,6 @@ object CommonUtil {
     }
   }
 
-  def getCsvHeaderColumns(
-      carbonLoadModel: CarbonLoadModel,
-      hadoopConf: Configuration,
-      staticPartitionCols: util.List[String] = new util.ArrayList[String]()): Array[String] = {
-    val delimiter = if (StringUtils.isEmpty(carbonLoadModel.getCsvDelimiter)) {
-      CarbonCommonConstants.COMMA
-    } else {
-      CarbonUtil.delimiterConverter(carbonLoadModel.getCsvDelimiter)
-    }
-    var csvFile: String = null
-    var csvHeader: String = carbonLoadModel.getCsvHeader
-    var csvColumns = if (StringUtils.isBlank(csvHeader)) {
-      // read header from csv file
-      csvFile = carbonLoadModel.getFactFilePath.split(",")(0)
-      csvHeader = CarbonUtil.readHeader(csvFile, hadoopConf)
-      if (StringUtils.isBlank(csvHeader)) {
-        throw new CarbonDataLoadingException("First line of the csv is not valid.")
-      }
-      csvHeader.toLowerCase().split(delimiter).map(_.replaceAll("\"", "").trim)
-    } else {
-      csvHeader.toLowerCase.split(CarbonCommonConstants.COMMA).map(_.trim)
-    }
-
-    if (!CarbonDataProcessorUtil.isHeaderValid(carbonLoadModel.getTableName, csvColumns,
-        carbonLoadModel.getCarbonDataLoadSchema, staticPartitionCols)) {
-      if (csvFile == null) {
-        LOGGER.error("CSV header in DDL is not proper."
-                     + " Column names in schema and CSV header are not the same.")
-        throw new CarbonDataLoadingException(
-          "CSV header in DDL is not proper. Column names in schema and CSV header are "
-          + "not the same.")
-      } else {
-        LOGGER.error(
-          "CSV header in input file is not proper. Column names in schema and csv header are not "
-          + "the same. Input file : " + CarbonUtil.removeAKSK(csvFile))
-        throw new CarbonDataLoadingException(
-          "CSV header in input file is not proper. Column names in schema and csv header are not "
-          + "the same. Input file : " + CarbonUtil.removeAKSK(csvFile))
-      }
-    }
-    // In case of static partition columns just change the name of header if already exists as
-    // we should not take the column from csv file and add them as new columns at the end.
-    if (staticPartitionCols.size() > 0) {
-      val scalaIgnoreColumns = staticPartitionCols.asScala
-      var updatedCols = csvColumns.map{col =>
-        if (scalaIgnoreColumns.exists(_.equalsIgnoreCase(col))) {
-          col + "1"
-        } else {
-          col
-        }
-      }.toList.asJava
-      updatedCols = new util.ArrayList[String](updatedCols)
-      updatedCols.addAll(staticPartitionCols)
-      updatedCols.asScala.toArray
-    } else {
-      csvColumns
-    }
-  }
-
   def validateMaxColumns(csvHeaders: Array[String], maxColumns: String): Int = {
     /*
     User configures both csvheadercolumns, maxcolumns,
@@ -880,6 +815,7 @@ object CommonUtil {
                 try {
                   val carbonTable = CarbonMetadata.getInstance
                     .getCarbonTable(identifier.getCarbonTableIdentifier.getTableUniqueName)
+                  SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable, true)
                   DataLoadingUtil.deleteLoadsAndUpdateMetadata(
                     isForceDeletion = true, carbonTable, null)
                 } catch {


[30/49] carbondata git commit: [HotFix][CheckStyle] Fix import related checkstyle

Posted by ja...@apache.org.
[HotFix][CheckStyle] Fix import related checkstyle

This closes #1952


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

Branch: refs/heads/carbonstore-rebase4
Commit: 5cc6d362395bcb20a81a19136fcf27b8bb0c45d9
Parents: e992013
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Thu Feb 8 15:39:45 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 00:30:22 2018 +0800

----------------------------------------------------------------------
 .../core/indexstore/blockletindex/BlockletDataRefNode.java         | 2 +-
 .../org/apache/carbondata/core/memory/HeapMemoryAllocator.java     | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/5cc6d362/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
index b8fd6ff..50862a7 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataRefNode.java
@@ -33,8 +33,8 @@ import org.apache.carbondata.core.datastore.chunk.reader.MeasureColumnChunkReade
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
 import org.apache.carbondata.core.indexstore.FineGrainBlocklet;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
-import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.metadata.blocklet.index.BlockletIndex;
+import org.apache.carbondata.core.util.BitSetGroup;
 
 /**
  * wrapper for blocklet data map data

http://git-wip-us.apache.org/repos/asf/carbondata/blob/5cc6d362/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index 2203b3b..5862933 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -17,11 +17,11 @@
 
 package org.apache.carbondata.core.memory;
 
-import javax.annotation.concurrent.GuardedBy;
 import java.lang.ref.WeakReference;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.Map;
+import javax.annotation.concurrent.GuardedBy;
 
 /**
  * Code ported from Apache Spark {org.apache.spark.unsafe.memory} package


[31/49] carbondata git commit: [CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

Posted by ja...@apache.org.
[CARBONDATA-2023][DataLoad] Add size base block allocation in data loading

Carbondata assign blocks to nodes at the beginning of data loading.
Previous block allocation strategy is block number based and it will
suffer skewed data problem if the size of input files differs a lot.

We introduced a size based block allocation strategy to optimize data
loading performance in skewed data scenario.

This closes #1808


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

Branch: refs/heads/carbonstore-rebase4
Commit: 43e34fceff7fb43f8435dcf0116c9b995f203a11
Parents: 806b984
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Thu Feb 8 14:42:39 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 00:30:50 2018 +0800

----------------------------------------------------------------------
 .../constants/CarbonLoadOptionConstants.java    |  10 +
 .../core/datastore/block/TableBlockInfo.java    |  29 ++
 .../carbondata/core/util/CarbonProperties.java  |  11 +
 docs/useful-tips-on-carbondata.md               |   1 +
 .../sdv/generated/MergeIndexTestCase.scala      |  11 +-
 .../CarbonIndexFileMergeTestCase.scala          |  19 +-
 .../StandardPartitionTableLoadingTestCase.scala |   2 +-
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |   4 +-
 .../spark/sql/hive/DistributionUtil.scala       |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  18 +-
 .../merger/NodeMultiBlockRelation.java          |  40 ++
 .../processing/util/CarbonLoaderUtil.java       | 494 ++++++++++++-------
 .../processing/util/CarbonLoaderUtilTest.java   | 125 +++++
 13 files changed, 563 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index bcfeba0..a6bf60f 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -114,4 +114,14 @@ public final class CarbonLoadOptionConstants {
    */
   public static final int MAX_EXTERNAL_DICTIONARY_SIZE = 10000000;
 
+  /**
+   * enable block size based block allocation while loading data. By default, carbondata assigns
+   * blocks to node based on block number. If this option is set to `true`, carbondata will
+   * consider block size first and make sure that all the nodes will process almost equal size of
+   * data. This option is especially useful when you encounter skewed data.
+   */
+  @CarbonProperty
+  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
+      = "carbon.load.skewedDataOptimization.enabled";
+  public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
index a7bfdba..c0cebe0 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/block/TableBlockInfo.java
@@ -19,6 +19,8 @@ package org.apache.carbondata.core.datastore.block;
 import java.io.IOException;
 import java.io.Serializable;
 import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -98,6 +100,20 @@ public class TableBlockInfo implements Distributable, Serializable {
 
   private String dataMapWriterPath;
 
+  /**
+   * comparator to sort by block size in descending order.
+   * Since each line is not exactly the same, the size of a InputSplit may differs,
+   * so we allow some deviation for these splits.
+   */
+  public static final Comparator<Distributable> DATA_SIZE_DESC_COMPARATOR =
+      new Comparator<Distributable>() {
+        @Override public int compare(Distributable o1, Distributable o2) {
+          long diff =
+              ((TableBlockInfo) o1).getBlockLength() - ((TableBlockInfo) o2).getBlockLength();
+          return diff < 0 ? 1 : (diff == 0 ? 0 : -1);
+        }
+      };
+
   public TableBlockInfo(String filePath, long blockOffset, String segmentId,
       String[] locations, long blockLength, ColumnarFormatVersion version,
       String[] deletedDeltaFilePath) {
@@ -434,4 +450,17 @@ public class TableBlockInfo implements Distributable, Serializable {
   public void setDataMapWriterPath(String dataMapWriterPath) {
     this.dataMapWriterPath = dataMapWriterPath;
   }
+
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder("TableBlockInfo{");
+    sb.append("filePath='").append(filePath).append('\'');
+    sb.append(", blockOffset=").append(blockOffset);
+    sb.append(", blockLength=").append(blockLength);
+    sb.append(", segmentId='").append(segmentId).append('\'');
+    sb.append(", blockletId='").append(blockletId).append('\'');
+    sb.append(", locations=").append(Arrays.toString(locations));
+    sb.append('}');
+    return sb.toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index 9d52669..474988c 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -1235,6 +1235,17 @@ public final class CarbonProperties {
       return CarbonCommonConstants.CARBON_SORT_TEMP_COMPRESSOR_DEFAULT;
     }
   }
+
+  /**
+   * whether optimization for skewed data is enabled
+   * @return true, if enabled; false for not enabled.
+   */
+  public boolean isLoadSkewedDataOptimizationEnabled() {
+    String skewedEnabled = getProperty(
+        CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION,
+        CarbonLoadOptionConstants.ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT);
+    return skewedEnabled.equalsIgnoreCase("true");
+  }
   /**
    * returns true if carbon property
    * @param key

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/docs/useful-tips-on-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/useful-tips-on-carbondata.md b/docs/useful-tips-on-carbondata.md
index 4d43003..ff339d0 100644
--- a/docs/useful-tips-on-carbondata.md
+++ b/docs/useful-tips-on-carbondata.md
@@ -169,5 +169,6 @@
   | carbon.use.local.dir | spark/carbonlib/carbon.properties | Data loading | Whether use YARN local directories for multi-table load disk load balance | If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance. |
   | carbon.use.multiple.temp.dir | spark/carbonlib/carbon.properties | Data loading | Whether to use multiple YARN local directories during table data loading for disk load balance | After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading. |
   | carbon.sort.temp.compressor | spark/carbonlib/carbon.properties | Data loading | Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading. | The optional values are 'SNAPPY','GZIP','BZIP2','LZ4' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck. |
+  | carbon.load.skewedDataOptimization.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable size based block allocation strategy for data loading. | When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB. |
 
   Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
index ed6d741..e2207bf 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/MergeIndexTestCase.scala
@@ -52,9 +52,8 @@ class MergeIndexTestCase extends QueryTest with BeforeAndAfterAll {
     sql(s"""LOAD DATA INPATH '$resourcesPath/Data/VmaLL100' INTO TABLE carbon_automation_merge 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_operaSysVe
 rsion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')""").collect
 
     val table = CarbonMetadata.getInstance().getCarbonTable("default","carbon_automation_merge")
-    val carbonTablePath = new CarbonTablePath(table.getCarbonTableIdentifier, table.getTablePath)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","0"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"0"), false)
     assert(getIndexFileCount("default", "carbon_automation_merge", "0") == 0)
     checkAnswer(sql("""Select count(*) from carbon_automation_nonmerge"""),
       sql("""Select count(*) from carbon_automation_merge"""))
@@ -71,11 +70,10 @@ class MergeIndexTestCase extends QueryTest with BeforeAndAfterAll {
     assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0") == 2)
     assert(getIndexFileCount("default", "carbon_automation_nonmerge", "1") == 2)
     val table = CarbonMetadata.getInstance().getCarbonTable("default","carbon_automation_nonmerge")
-    val carbonTablePath = new CarbonTablePath(table.getCarbonTableIdentifier, table.getTablePath)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","0"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"0"), false)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","1"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"1"), false)
     assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0") == 0)
     assert(getIndexFileCount("default", "carbon_automation_nonmerge", "1") == 0)
     checkAnswer(sql("""Select count(*) from carbon_automation_nonmerge"""), rows)
@@ -95,9 +93,8 @@ class MergeIndexTestCase extends QueryTest with BeforeAndAfterAll {
     assert(getIndexFileCount("default", "carbon_automation_nonmerge", "1") == 2)
     sql("ALTER TABLE carbon_automation_nonmerge COMPACT 'minor'").collect()
     val table = CarbonMetadata.getInstance().getCarbonTable("default","carbon_automation_nonmerge")
-    val carbonTablePath = new CarbonTablePath(table.getCarbonTableIdentifier, table.getTablePath)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","0.1"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"0.1"), false)
     assert(getIndexFileCount("default", "carbon_automation_nonmerge", "0.1") == 0)
     checkAnswer(sql("""Select count(*) from carbon_automation_nonmerge"""), rows)
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
index 895b0b5..bc22138 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CarbonIndexFileMergeTestCase.scala
@@ -62,9 +62,8 @@ class CarbonIndexFileMergeTestCase
     sql(s"LOAD DATA LOCAL INPATH '$file2' INTO TABLE indexmerge OPTIONS('header'='false', " +
         s"'GLOBAL_SORT_PARTITIONS'='100')")
     val table = CarbonMetadata.getInstance().getCarbonTable("default","indexmerge")
-    val carbonTablePath = new CarbonTablePath(table.getCarbonTableIdentifier, table.getTablePath)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","0"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"0"), false)
     assert(getIndexFileCount("default_indexmerge", "0") == 0)
     checkAnswer(sql("""Select count(*) from nonindexmerge"""),
       sql("""Select count(*) from indexmerge"""))
@@ -86,11 +85,10 @@ class CarbonIndexFileMergeTestCase
     assert(getIndexFileCount("default_nonindexmerge", "0") == 100)
     assert(getIndexFileCount("default_nonindexmerge", "1") == 100)
     val table = CarbonMetadata.getInstance().getCarbonTable("default","nonindexmerge")
-    val carbonTablePath = new CarbonTablePath(table.getCarbonTableIdentifier, table.getTablePath)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","0"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"0"), false)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","1"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"1"), false)
     assert(getIndexFileCount("default_nonindexmerge", "0") == 0)
     assert(getIndexFileCount("default_nonindexmerge", "1") == 0)
     checkAnswer(sql("""Select count(*) from nonindexmerge"""), rows)
@@ -112,11 +110,10 @@ class CarbonIndexFileMergeTestCase
     assert(getIndexFileCount("default_nonindexmerge", "0") == 100)
     assert(getIndexFileCount("default_nonindexmerge", "1") == 100)
     val table = CarbonMetadata.getInstance().getCarbonTable("default","nonindexmerge")
-    val carbonTablePath = new CarbonTablePath(table.getCarbonTableIdentifier, table.getTablePath)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","0"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"0"), false)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","1"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"1"), false)
     assert(getIndexFileCount("default_nonindexmerge", "0") == 0)
     assert(getIndexFileCount("default_nonindexmerge", "1") == 0)
     checkAnswer(sql("""Select count(*) from nonindexmerge"""), rows)
@@ -142,9 +139,8 @@ class CarbonIndexFileMergeTestCase
     assert(getIndexFileCount("default_nonindexmerge", "1") == 100)
     sql("ALTER TABLE nonindexmerge COMPACT 'minor'").collect()
     val table = CarbonMetadata.getInstance().getCarbonTable("default","nonindexmerge")
-    val carbonTablePath = new CarbonTablePath(table.getCarbonTableIdentifier, table.getTablePath)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","0.1"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"0.1"), false)
     assert(getIndexFileCount("default_nonindexmerge", "0.1") == 0)
     checkAnswer(sql("""Select count(*) from nonindexmerge"""), rows)
   }
@@ -172,9 +168,8 @@ class CarbonIndexFileMergeTestCase
     assert(getIndexFileCount("default_nonindexmerge", "3") == 100)
     sql("ALTER TABLE nonindexmerge COMPACT 'minor'").collect()
     val table = CarbonMetadata.getInstance().getCarbonTable("default","nonindexmerge")
-    val carbonTablePath = new CarbonTablePath(table.getCarbonTableIdentifier, table.getTablePath)
     new CarbonIndexFileMergeWriter()
-      .mergeCarbonIndexFilesOfSegment(carbonTablePath.getSegmentDir("0","0.1"), false)
+      .mergeCarbonIndexFilesOfSegment(CarbonTablePath.getSegmentPath(table.getTablePath,"0.1"), false)
     assert(getIndexFileCount("default_nonindexmerge", "0") == 100)
     assert(getIndexFileCount("default_nonindexmerge", "1") == 100)
     assert(getIndexFileCount("default_nonindexmerge", "2") == 100)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
index 2ce46ef..baf1627 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionTableLoadingTestCase.scala
@@ -35,7 +35,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.metadata.{CarbonMetadata, SegmentFileStore}
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
 
 class StandardPartitionTableLoadingTestCase extends QueryTest with BeforeAndAfterAll {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 06acbba..8ba2767 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -281,8 +281,10 @@ class NewCarbonDataLoadRDD[K, V](
         val format = new CSVInputFormat
 
         val split = theSplit.asInstanceOf[CarbonNodePartition]
+        val inputSize = split.blocksDetails.map(_.getBlockLength).sum * 0.1 * 10  / 1024 / 1024
         logInfo("Input split: " + split.serializableHadoopSplit)
-        logInfo("The Block Count in this node :" + split.nodeBlocksDetail.length)
+        logInfo("The block count in this node: " + split.nodeBlocksDetail.length)
+        logInfo(f"The input data size in this node: $inputSize%.2fMB")
         CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordHostBlockMap(
             split.serializableHadoopSplit, split.nodeBlocksDetail.length)
         carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
index 1958d61..a676dd8 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/hive/DistributionUtil.scala
@@ -128,7 +128,7 @@ object DistributionUtil {
    */
   def ensureExecutorsAndGetNodeList(blockList: Seq[Distributable],
       sparkContext: SparkContext): Seq[String] = {
-    val nodeMapping = CarbonLoaderUtil.getRequiredExecutors(blockList.asJava)
+    val nodeMapping = CarbonLoaderUtil.nodeBlockMapping(blockList.asJava)
     ensureExecutorsByNumberAndGetNodeList(nodeMapping, blockList, sparkContext)
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 8d3110a..acde951 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -1026,10 +1026,16 @@ object CarbonDataRDDFactory {
     val startTime = System.currentTimeMillis
     val activeNodes = DistributionUtil
       .ensureExecutorsAndGetNodeList(blockList, sqlContext.sparkContext)
-    val nodeBlockMapping =
-      CarbonLoaderUtil
-        .nodeBlockMapping(blockList.toSeq.asJava, -1, activeNodes.toList.asJava).asScala
-        .toSeq
+    val skewedDataOptimization = CarbonProperties.getInstance()
+      .isLoadSkewedDataOptimizationEnabled()
+    val blockAssignStrategy = if (skewedDataOptimization) {
+      CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_SIZE_FIRST
+    } else {
+      CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_NUM_FIRST
+    }
+    LOGGER.info(s"Allocating block to nodes using strategy: $blockAssignStrategy")
+    val nodeBlockMapping = CarbonLoaderUtil.nodeBlockMapping(blockList.toSeq.asJava, -1,
+      activeNodes.toList.asJava, blockAssignStrategy).asScala.toSeq
     val timeElapsed: Long = System.currentTimeMillis - startTime
     LOGGER.info("Total Time taken in block allocation: " + timeElapsed)
     LOGGER.info(s"Total no of blocks: ${ blockList.length }, " +
@@ -1037,7 +1043,9 @@ object CarbonDataRDDFactory {
     var str = ""
     nodeBlockMapping.foreach { entry =>
       val tableBlock = entry._2
-      str = str + "#Node: " + entry._1 + " no.of.blocks: " + tableBlock.size()
+      val totalSize = tableBlock.asScala.map(_.asInstanceOf[TableBlockInfo].getBlockLength).sum
+      str = str + "#Node: " + entry._1 + ", no.of.blocks: " + tableBlock.size() +
+            f", totalsize.of.blocks: ${totalSize * 0.1 * 10 / 1024 /1024}%.2fMB"
       tableBlock.asScala.foreach(tableBlockInfo =>
         if (!tableBlockInfo.getLocations.exists(hostentry =>
           hostentry.equalsIgnoreCase(entry._1)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java b/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
index ec2ddaf..1bb5736 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/NodeMultiBlockRelation.java
@@ -16,15 +16,41 @@
  */
 package org.apache.carbondata.processing.merger;
 
+import java.util.Comparator;
 import java.util.List;
 
 import org.apache.carbondata.core.datastore.block.Distributable;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 
 public class NodeMultiBlockRelation implements Comparable<NodeMultiBlockRelation> {
 
   private final List<Distributable> blocks;
   private final String node;
 
+  /**
+   * comparator to sort by data size in descending order. This is used to assign big blocks to
+   * bigger nodes first.
+   */
+  public static final Comparator<NodeMultiBlockRelation> DATA_SIZE_DESC_COMPARATOR =
+      new Comparator<NodeMultiBlockRelation>() {
+        @Override
+        public int compare(NodeMultiBlockRelation o1, NodeMultiBlockRelation o2) {
+          long diff = o1.getTotalSizeOfBlocks() - o2.getTotalSizeOfBlocks();
+          return diff > 0 ? -1 : (diff < 0 ? 1 : 0);
+        }
+      };
+  /**
+   * comparator to sort by data size in ascending order. This is used to assign left over blocks to
+   * smaller nodes first.
+   */
+  public static final Comparator<NodeMultiBlockRelation> DATA_SIZE_ASC_COMPARATOR =
+      new Comparator<NodeMultiBlockRelation>() {
+        @Override
+        public int compare(NodeMultiBlockRelation o1, NodeMultiBlockRelation o2) {
+          long diff = o1.getTotalSizeOfBlocks() - o2.getTotalSizeOfBlocks();
+          return diff > 0 ? 1 : (diff < 0 ? -1 : 0);
+        }
+      };
   public NodeMultiBlockRelation(String node, List<Distributable> blocks) {
     this.node = node;
     this.blocks = blocks;
@@ -39,6 +65,20 @@ public class NodeMultiBlockRelation implements Comparable<NodeMultiBlockRelation
     return node;
   }
 
+  /**
+   * get the total size of the blocks
+   * @return size in bytes
+   */
+  public long getTotalSizeOfBlocks() {
+    long totalSize = 0;
+    if (blocks.get(0) instanceof TableBlockInfo) {
+      for (Distributable block : blocks) {
+        totalSize += ((TableBlockInfo) block).getBlockLength();
+      }
+    }
+    return totalSize;
+  }
+
   @Override public int compareTo(NodeMultiBlockRelation obj) {
     return this.blocks.size() - obj.getBlocks().size();
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index 03c8c27..1eea61d 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -24,7 +24,16 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.charset.Charset;
 import java.text.SimpleDateFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -36,6 +45,7 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.Segment;
 import org.apache.carbondata.core.datastore.block.Distributable;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
@@ -56,7 +66,6 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-import org.apache.carbondata.processing.merger.NodeBlockRelation;
 import org.apache.carbondata.processing.merger.NodeMultiBlockRelation;
 import static org.apache.carbondata.core.enums.EscapeSequences.*;
 
@@ -71,6 +80,23 @@ public final class CarbonLoaderUtil {
   private CarbonLoaderUtil() {
   }
 
+  /**
+   * strategy for assign blocks to nodes/executors
+   */
+  public enum BlockAssignmentStrategy {
+    BLOCK_NUM_FIRST("Assign blocks to node base on number of blocks"),
+    BLOCK_SIZE_FIRST("Assign blocks to node base on data size of blocks");
+    private String name;
+    BlockAssignmentStrategy(String name) {
+      this.name = name;
+    }
+
+    @Override
+    public String toString() {
+      return this.getClass().getSimpleName() + ':' + this.name;
+    }
+  }
+
   public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
     String segmentPath = CarbonTablePath.getSegmentPath(
         loadModel.getTablePath(), currentLoad + "");
@@ -492,9 +518,9 @@ public final class CarbonLoaderUtil {
   public static Map<String, List<List<Distributable>>> nodeBlockTaskMapping(
       List<Distributable> blockInfos, int noOfNodesInput, int parallelism,
       List<String> activeNode) {
-
     Map<String, List<Distributable>> mapOfNodes =
-        CarbonLoaderUtil.nodeBlockMapping(blockInfos, noOfNodesInput, activeNode);
+        CarbonLoaderUtil.nodeBlockMapping(blockInfos, noOfNodesInput, activeNode,
+            BlockAssignmentStrategy.BLOCK_NUM_FIRST);
     int taskPerNode = parallelism / mapOfNodes.size();
     //assigning non zero value to noOfTasksPerNode
     int noOfTasksPerNode = taskPerNode == 0 ? 1 : taskPerNode;
@@ -510,7 +536,8 @@ public final class CarbonLoaderUtil {
    */
   public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos,
       int noOfNodesInput) {
-    return nodeBlockMapping(blockInfos, noOfNodesInput, null);
+    return nodeBlockMapping(blockInfos, noOfNodesInput, null,
+        BlockAssignmentStrategy.BLOCK_NUM_FIRST);
   }
 
   /**
@@ -525,82 +552,59 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * the method returns the number of required executors
-   *
-   * @param blockInfos
-   * @return
-   */
-  public static Map<String, List<Distributable>> getRequiredExecutors(
-      List<Distributable> blockInfos) {
-    List<NodeBlockRelation> flattenedList =
-        new ArrayList<NodeBlockRelation>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (Distributable blockInfo : blockInfos) {
-      try {
-        for (String eachNode : blockInfo.getLocations()) {
-          NodeBlockRelation nbr = new NodeBlockRelation(blockInfo, eachNode);
-          flattenedList.add(nbr);
-        }
-      } catch (IOException e) {
-        throw new RuntimeException("error getting location of block: " + blockInfo.toString(), e);
-      }
-    }
-    // sort the flattened data.
-    Collections.sort(flattenedList);
-    Map<String, List<Distributable>> nodeAndBlockMapping =
-        new LinkedHashMap<String, List<Distributable>>(
-            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    // from the flattened list create a mapping of node vs Data blocks.
-    createNodeVsBlockMapping(flattenedList, nodeAndBlockMapping);
-    return nodeAndBlockMapping;
-  }
-
-  /**
    * This method will divide the blocks among the nodes as per the data locality
    *
-   * @param blockInfos
+   * @param blockInfos blocks
    * @param noOfNodesInput -1 if number of nodes has to be decided
    *                       based on block location information
-   * @return
+   * @param blockAssignmentStrategy strategy used to assign blocks
+   * @return a map that maps node to blocks
    */
-  public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos,
-      int noOfNodesInput, List<String> activeNodes) {
-
-    Map<String, List<Distributable>> nodeBlocksMap =
-        new HashMap<String, List<Distributable>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    List<NodeBlockRelation> flattenedList =
-        new ArrayList<NodeBlockRelation>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    Set<Distributable> uniqueBlocks =
-        new HashSet<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    Set<String> nodes = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    createFlattenedListFromMap(blockInfos, flattenedList, uniqueBlocks, nodes);
+  public static Map<String, List<Distributable>> nodeBlockMapping(
+      List<Distributable> blockInfos, int noOfNodesInput, List<String> activeNodes,
+      BlockAssignmentStrategy blockAssignmentStrategy) {
+    ArrayList<NodeMultiBlockRelation> rtnNode2Blocks = new ArrayList<>();
+
+    Set<Distributable> uniqueBlocks = new HashSet<>(blockInfos);
+    ArrayList<NodeMultiBlockRelation> originNode2Blocks = createNode2BlocksMapping(blockInfos);
+    Set<String> nodes = new HashSet<>(originNode2Blocks.size());
+    for (NodeMultiBlockRelation relation : originNode2Blocks) {
+      nodes.add(relation.getNode());
+    }
 
     int noofNodes = (-1 == noOfNodesInput) ? nodes.size() : noOfNodesInput;
     if (null != activeNodes) {
       noofNodes = activeNodes.size();
     }
-    int blocksPerNode = blockInfos.size() / noofNodes;
-    blocksPerNode = blocksPerNode <= 0 ? 1 : blocksPerNode;
 
-    // sort the flattened data.
-    Collections.sort(flattenedList);
-
-    Map<String, List<Distributable>> nodeAndBlockMapping =
-        new LinkedHashMap<String, List<Distributable>>(
-            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    // from the flattened list create a mapping of node vs Data blocks.
-    createNodeVsBlockMapping(flattenedList, nodeAndBlockMapping);
+    // calculate the average expected size for each node
+    long sizePerNode = 0;
+    if (BlockAssignmentStrategy.BLOCK_NUM_FIRST == blockAssignmentStrategy) {
+      sizePerNode = blockInfos.size() / noofNodes;
+      sizePerNode = sizePerNode <= 0 ? 1 : sizePerNode;
+    } else if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
+      long totalFileSize = 0;
+      for (Distributable blockInfo : uniqueBlocks) {
+        totalFileSize += ((TableBlockInfo) blockInfo).getBlockLength();
+      }
+      sizePerNode = totalFileSize / noofNodes;
+    }
 
-    // so now we have a map of node vs blocks. allocate the block as per the order
-    createOutputMap(nodeBlocksMap, blocksPerNode, uniqueBlocks, nodeAndBlockMapping, activeNodes);
+    // assign blocks to each node
+    assignBlocksByDataLocality(rtnNode2Blocks, sizePerNode, uniqueBlocks, originNode2Blocks,
+        activeNodes, blockAssignmentStrategy);
 
     // if any blocks remain then assign them to nodes in round robin.
-    assignLeftOverBlocks(nodeBlocksMap, uniqueBlocks, blocksPerNode, activeNodes);
+    assignLeftOverBlocks(rtnNode2Blocks, uniqueBlocks, sizePerNode, activeNodes,
+        blockAssignmentStrategy);
 
-    return nodeBlocksMap;
+    // convert
+    Map<String, List<Distributable>> rtnNodeBlocksMap =
+        new HashMap<String, List<Distributable>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    for (NodeMultiBlockRelation relation : rtnNode2Blocks) {
+      rtnNodeBlocksMap.put(relation.getNode(), relation.getBlocks());
+    }
+    return rtnNodeBlocksMap;
   }
 
   /**
@@ -675,92 +679,207 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * If any left over data blocks are present then assign those to nodes in round robin way.
-   *
-   * @param outputMap
-   * @param uniqueBlocks
+   * If any left over data blocks are present then assign those to nodes in round robin way. This
+   * will not obey the data locality.
    */
-  private static void assignLeftOverBlocks(Map<String, List<Distributable>> outputMap,
-      Set<Distributable> uniqueBlocks, int noOfBlocksPerNode, List<String> activeNodes) {
+  private static void assignLeftOverBlocks(ArrayList<NodeMultiBlockRelation> outputMap,
+      Set<Distributable> leftOverBlocks, long expectedSizePerNode, List<String> activeNodes,
+      BlockAssignmentStrategy blockAssignmentStrategy) {
+    Map<String, Integer> node2Idx = new HashMap<>(outputMap.size());
+    for (int idx = 0; idx < outputMap.size(); idx++) {
+      node2Idx.put(outputMap.get(idx).getNode(), idx);
+    }
 
+    // iterate all the nodes and try to allocate blocks to the nodes
     if (activeNodes != null) {
       for (String activeNode : activeNodes) {
-        List<Distributable> blockLst = outputMap.get(activeNode);
-        if (null == blockLst) {
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("Second assignment iteration: assign for executor: " + activeNode);
+        }
+
+        Integer idx;
+        List<Distributable> blockLst;
+        if (node2Idx.containsKey(activeNode)) {
+          idx = node2Idx.get(activeNode);
+          blockLst = outputMap.get(idx).getBlocks();
+        } else {
+          idx = node2Idx.size();
           blockLst = new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
         }
-        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
-        if (blockLst.size() > 0) {
-          outputMap.put(activeNode, blockLst);
+        populateBlocks(leftOverBlocks, expectedSizePerNode, blockLst, blockAssignmentStrategy);
+
+        if (!node2Idx.containsKey(activeNode) && blockLst.size() > 0) {
+          outputMap.add(idx, new NodeMultiBlockRelation(activeNode, blockLst));
+          node2Idx.put(activeNode, idx);
         }
       }
     } else {
-      for (Map.Entry<String, List<Distributable>> entry : outputMap.entrySet()) {
-        List<Distributable> blockLst = entry.getValue();
-        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
+      for (NodeMultiBlockRelation entry : outputMap) {
+        List<Distributable> blockLst = entry.getBlocks();
+        populateBlocks(leftOverBlocks, expectedSizePerNode, blockLst, blockAssignmentStrategy);
       }
-
     }
 
-    for (Map.Entry<String, List<Distributable>> entry : outputMap.entrySet()) {
-      Iterator<Distributable> blocks = uniqueBlocks.iterator();
-      if (blocks.hasNext()) {
-        Distributable block = blocks.next();
-        List<Distributable> blockLst = entry.getValue();
-        blockLst.add(block);
-        blocks.remove();
-      }
+    // if there is still blocks left, allocate them in round robin manner to each nodes
+    assignBlocksUseRoundRobin(outputMap, leftOverBlocks, blockAssignmentStrategy);
+  }
+
+  /**
+   * assign remaining blocks to nodes
+   *
+   * @param remainingBlocks blocks to be allocated
+   * @param expectedSizePerNode expected size for each node
+   * @param blockLst destination for the blocks to be allocated
+   * @param blockAssignmentStrategy block assignment stretegy
+   */
+  private static void populateBlocks(Set<Distributable> remainingBlocks,
+      long expectedSizePerNode, List<Distributable> blockLst,
+      BlockAssignmentStrategy blockAssignmentStrategy) {
+    switch (blockAssignmentStrategy) {
+      case BLOCK_NUM_FIRST:
+        populateBlocksByNum(remainingBlocks, expectedSizePerNode, blockLst);
+        break;
+      case BLOCK_SIZE_FIRST:
+        populateBlocksBySize(remainingBlocks, expectedSizePerNode, blockLst);
+        break;
+      default:
+        throw new IllegalArgumentException(
+            "Unsupported block assignment strategy: " + blockAssignmentStrategy);
     }
   }
 
   /**
-   * The method populate the blockLst to be allocate to a specific node.
-   * @param uniqueBlocks
-   * @param noOfBlocksPerNode
-   * @param blockLst
+   * Taken N number of distributable blocks from {@param remainingBlocks} and add them to output
+   * {@param blockLst}. After added, the total number of {@param blockLst} is less
+   * than {@param expectedSizePerNode}.
    */
-  private static void populateBlocks(Set<Distributable> uniqueBlocks, int noOfBlocksPerNode,
-      List<Distributable> blockLst) {
-    Iterator<Distributable> blocks = uniqueBlocks.iterator();
-    //if the node is already having the per block nodes then avoid assign the extra blocks
-    if (blockLst.size() == noOfBlocksPerNode) {
+  private static void populateBlocksByNum(Set<Distributable> remainingBlocks,
+      long expectedSizePerNode, List<Distributable> blockLst) {
+    Iterator<Distributable> blocks = remainingBlocks.iterator();
+    // if the node is already having the per block nodes then avoid assign the extra blocks
+    if (blockLst.size() == expectedSizePerNode) {
       return;
     }
     while (blocks.hasNext()) {
       Distributable block = blocks.next();
       blockLst.add(block);
       blocks.remove();
-      if (blockLst.size() >= noOfBlocksPerNode) {
+      if (blockLst.size() >= expectedSizePerNode) {
         break;
       }
     }
   }
 
   /**
-   * To create the final output of the Node and Data blocks
-   *
-   * @param outputMap
-   * @param blocksPerNode
-   * @param uniqueBlocks
-   * @param nodeAndBlockMapping
-   * @param activeNodes
+   * Taken N number of distributable blocks from {@param remainingBlocks} and add them to output
+   * {@param blockLst}. After added, the total accumulated block size of {@param blockLst}
+   * is less than {@param expectedSizePerNode}.
    */
-  private static void createOutputMap(Map<String, List<Distributable>> outputMap, int blocksPerNode,
-      Set<Distributable> uniqueBlocks, Map<String, List<Distributable>> nodeAndBlockMapping,
-      List<String> activeNodes) {
+  private static void populateBlocksBySize(Set<Distributable> remainingBlocks,
+      long expectedSizePerNode, List<Distributable> blockLst) {
+    Iterator<Distributable> blocks = remainingBlocks.iterator();
+    //if the node is already having the avg node size then avoid assign the extra blocks
+    long fileSize = 0;
+    for (Distributable block : blockLst) {
+      fileSize += ((TableBlockInfo) block).getBlockLength();
+    }
+    if (fileSize >= expectedSizePerNode) {
+      LOGGER.debug("Capacity is full, skip allocate blocks on this node");
+      return;
+    }
 
-    ArrayList<NodeMultiBlockRelation> multiBlockRelations =
-        new ArrayList<>(nodeAndBlockMapping.size());
-    for (Map.Entry<String, List<Distributable>> entry : nodeAndBlockMapping.entrySet()) {
-      multiBlockRelations.add(new NodeMultiBlockRelation(entry.getKey(), entry.getValue()));
+    while (blocks.hasNext()) {
+      Distributable block = blocks.next();
+      long thisBlockSize = ((TableBlockInfo) block).getBlockLength();
+      if (fileSize < expectedSizePerNode) {
+        // `fileSize==0` means there are no blocks assigned to this node before
+        if (fileSize == 0 || fileSize + thisBlockSize <= expectedSizePerNode * 1.1D) {
+          blockLst.add(block);
+          if (LOGGER.isDebugEnabled()) {
+            LOGGER.debug("Second Assignment iteration: "
+                + ((TableBlockInfo) block).getFilePath() + "-"
+                + ((TableBlockInfo) block).getBlockLength() + "-->currentNode");
+          }
+          fileSize += thisBlockSize;
+          blocks.remove();
+        }
+      } else {
+        break;
+      }
     }
-    // sort nodes based on number of blocks per node, so that nodes having lesser blocks
-    // are assigned first
-    Collections.sort(multiBlockRelations);
+  }
 
-    for (NodeMultiBlockRelation nodeMultiBlockRelation : multiBlockRelations) {
+  /**
+   * allocate the blocks in round robin manner
+   */
+  private static void assignBlocksUseRoundRobin(ArrayList<NodeMultiBlockRelation> node2Blocks,
+      Set<Distributable> remainingBlocks, BlockAssignmentStrategy blockAssignmentStrategy) {
+    switch (blockAssignmentStrategy) {
+      case BLOCK_NUM_FIRST:
+        roundRobinAssignBlocksByNum(node2Blocks, remainingBlocks);
+        break;
+      case BLOCK_SIZE_FIRST:
+        roundRobinAssignBlocksBySize(node2Blocks, remainingBlocks);
+        break;
+      default:
+        throw new IllegalArgumentException("Unsupported block assignment strategy: "
+            + blockAssignmentStrategy);
+    }
+  }
+
+  private static void roundRobinAssignBlocksByNum(ArrayList<NodeMultiBlockRelation> outputMap,
+      Set<Distributable> remainingBlocks) {
+    for (NodeMultiBlockRelation relation: outputMap) {
+      Iterator<Distributable> blocks = remainingBlocks.iterator();
+      if (blocks.hasNext()) {
+        Distributable block = blocks.next();
+        List<Distributable> blockLst = relation.getBlocks();
+        blockLst.add(block);
+        blocks.remove();
+      }
+    }
+  }
+
+  private static void roundRobinAssignBlocksBySize(ArrayList<NodeMultiBlockRelation> outputMap,
+      Set<Distributable> remainingBlocks) {
+    Iterator<Distributable> blocks = remainingBlocks.iterator();
+    while (blocks.hasNext()) {
+      // sort the allocated node-2-blocks in ascending order, the total data size of first one is
+      // the smallest, so we assign this block to it.
+      Collections.sort(outputMap, NodeMultiBlockRelation.DATA_SIZE_ASC_COMPARATOR);
+      Distributable block = blocks.next();
+      List<Distributable> blockLst = outputMap.get(0).getBlocks();
+      blockLst.add(block);
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("RoundRobin assignment iteration: "
+            + ((TableBlockInfo) block).getFilePath() + "-"
+            + ((TableBlockInfo) block).getBlockLength() + "-->" + outputMap.get(0).getNode());
+      }
+      blocks.remove();
+    }
+  }
+  /**
+   * allocate distributable blocks to nodes based on data locality
+   */
+  private static void assignBlocksByDataLocality(
+      ArrayList<NodeMultiBlockRelation> outputNode2Blocks,
+      long expectedSizePerNode, Set<Distributable> remainingBlocks,
+      List<NodeMultiBlockRelation> inputNode2Blocks, List<String> activeNodes,
+      BlockAssignmentStrategy blockAssignmentStrategy) {
+    if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
+      // sort nodes based on data size of all blocks per node, so that nodes having bigger size
+      // are assigned first
+      Collections.sort(inputNode2Blocks, NodeMultiBlockRelation.DATA_SIZE_DESC_COMPARATOR);
+    } else {
+      // sort nodes based on number of blocks per node, so that nodes having lesser blocks
+      // are assigned first
+      Collections.sort(inputNode2Blocks);
+    }
+
+    Map<String, Integer> executor2Idx = new HashMap<>();
+    for (NodeMultiBlockRelation nodeMultiBlockRelation : inputNode2Blocks) {
       String nodeName = nodeMultiBlockRelation.getNode();
-      //assign the block to the node only if the node is active
+      // assign the block to the node only if the node is active
       String activeExecutor = nodeName;
       if (null != activeNodes) {
         activeExecutor = getActiveExecutor(activeNodes, nodeName);
@@ -768,29 +887,75 @@ public final class CarbonLoaderUtil {
           continue;
         }
       }
-      // this loop will be for each NODE
-      int nodeCapacity = 0;
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("First Assignment iteration: assign for executor: " + activeExecutor);
+      }
+
+      List<Distributable> blocksInThisNode = nodeMultiBlockRelation.getBlocks();
+      if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
+        // sort blocks based on block size, so that bigger blocks will be assigned first
+        Collections.sort(blocksInThisNode, TableBlockInfo.DATA_SIZE_DESC_COMPARATOR);
+      }
+
+      long nodeCapacity = 0;
       // loop thru blocks of each Node
       for (Distributable block : nodeMultiBlockRelation.getBlocks()) {
+        if (!remainingBlocks.contains(block)) {
+          // this block has been added before
+          continue;
+        }
+        // this is the first time to add block to this node, initialize it
+        if (!executor2Idx.containsKey(activeExecutor)) {
+          Integer idx = executor2Idx.size();
+          outputNode2Blocks.add(idx, new NodeMultiBlockRelation(activeExecutor,
+              new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE)));
+          executor2Idx.put(activeExecutor, idx);
+        }
 
-        // check if this is already assigned.
-        if (uniqueBlocks.contains(block)) {
-
-          if (null == outputMap.get(activeExecutor)) {
-            List<Distributable> list =
-                new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-            outputMap.put(activeExecutor, list);
-          }
-          // assign this block to this node if node has capacity left
-          if (nodeCapacity < blocksPerNode) {
-            List<Distributable> infos = outputMap.get(activeExecutor);
+        // assign this block to this node if node has capacity left
+        if (BlockAssignmentStrategy.BLOCK_NUM_FIRST == blockAssignmentStrategy) {
+          if (nodeCapacity < expectedSizePerNode) {
+            Integer idx = executor2Idx.get(activeExecutor);
+            List<Distributable> infos = outputNode2Blocks.get(idx).getBlocks();
             infos.add(block);
             nodeCapacity++;
-            uniqueBlocks.remove(block);
+            if (LOGGER.isDebugEnabled()) {
+              LOGGER.debug(
+                  "First Assignment iteration: " + ((TableBlockInfo) block).getFilePath() + '-'
+                      + ((TableBlockInfo) block).getBlockLength() + "-->" + activeExecutor);
+            }
+            remainingBlocks.remove(block);
+          } else {
+            // No need to continue loop as node is full
+            break;
+          }
+        } else if (BlockAssignmentStrategy.BLOCK_SIZE_FIRST == blockAssignmentStrategy) {
+          long thisBlockSize = ((TableBlockInfo) block).getBlockLength();
+          // `nodeCapacity == 0` means that there is a huge block that already exceed the
+          // `expectedSize` of the node, so we have to assign it to some node, otherwise it will
+          // be assigned in the last RoundRobin iteration.
+          if (nodeCapacity == 0 || nodeCapacity < expectedSizePerNode) {
+            if (nodeCapacity == 0 || nodeCapacity + thisBlockSize <= expectedSizePerNode * 1.05D) {
+              Integer idx = executor2Idx.get(activeExecutor);
+              List<Distributable> blocks = outputNode2Blocks.get(idx).getBlocks();
+              blocks.add(block);
+              nodeCapacity += thisBlockSize;
+              if (LOGGER.isDebugEnabled()) {
+                LOGGER.debug(
+                    "First Assignment iteration: " + ((TableBlockInfo) block).getFilePath() + '-'
+                        + ((TableBlockInfo) block).getBlockLength() + "-->" + activeExecutor);
+              }
+              remainingBlocks.remove(block);
+            }
+            // this block is too big for current node and there are still capacity left
+            // for small files, so continue to allocate block on this node in next iteration.
           } else {
             // No need to continue loop as node is full
             break;
           }
+        } else {
+          throw new IllegalArgumentException(
+              "Unsupported block assignment strategy: " + blockAssignmentStrategy);
         }
       }
     }
@@ -834,60 +999,37 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * Create the Node and its related blocks Mapping and put in a Map
+   * Create node to blocks mapping
    *
-   * @param flattenedList
-   * @param nodeAndBlockMapping
+   * @param blockInfos input block info
    */
-  private static void createNodeVsBlockMapping(List<NodeBlockRelation> flattenedList,
-      Map<String, List<Distributable>> nodeAndBlockMapping) {
-    for (NodeBlockRelation nbr : flattenedList) {
-      String node = nbr.getNode();
-      List<Distributable> list;
-
-      if (null == nodeAndBlockMapping.get(node)) {
-        list = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        list.add(nbr.getBlock());
-        nodeAndBlockMapping.put(node, list);
-      } else {
-        list = nodeAndBlockMapping.get(node);
-        list.add(nbr.getBlock());
-      }
-    }
-    /*for resolving performance issue, removed values() with entrySet () iterating the values and
-    sorting it.entrySet will give the logical view for hashMap and we dont query the map twice for
-    each key whereas values () iterate twice*/
-    Iterator<Map.Entry<String, List<Distributable>>> iterator =
-        nodeAndBlockMapping.entrySet().iterator();
-    while (iterator.hasNext()) {
-      Collections.sort(iterator.next().getValue());
-    }
-  }
+  private static ArrayList<NodeMultiBlockRelation> createNode2BlocksMapping(
+      List<Distributable> blockInfos) {
+    Map<String, Integer> node2Idx = new HashMap<>();
+    ArrayList<NodeMultiBlockRelation> node2Blocks = new ArrayList<>();
 
-  /**
-   * Create the flat List i.e flattening of the Map.
-   *
-   * @param blockInfos
-   * @param flattenedList
-   * @param uniqueBlocks
-   */
-  private static void createFlattenedListFromMap(List<Distributable> blockInfos,
-      List<NodeBlockRelation> flattenedList, Set<Distributable> uniqueBlocks,
-      Set<String> nodeList) {
     for (Distributable blockInfo : blockInfos) {
-      // put the blocks in the set
-      uniqueBlocks.add(blockInfo);
-
       try {
-        for (String eachNode : blockInfo.getLocations()) {
-          NodeBlockRelation nbr = new NodeBlockRelation(blockInfo, eachNode);
-          flattenedList.add(nbr);
-          nodeList.add(eachNode);
+        for (final String eachNode : blockInfo.getLocations()) {
+          if (node2Idx.containsKey(eachNode)) {
+            Integer idx = node2Idx.get(eachNode);
+            List<Distributable> blocks = node2Blocks.get(idx).getBlocks();
+            blocks.add(blockInfo);
+          } else {
+            // add blocks to this node for the first time
+            Integer idx = node2Idx.size();
+            List<Distributable> blocks = new ArrayList<>();
+            blocks.add(blockInfo);
+            node2Blocks.add(idx, new NodeMultiBlockRelation(eachNode, blocks));
+            node2Idx.put(eachNode, idx);
+          }
         }
       } catch (IOException e) {
         throw new RuntimeException("error getting location of block: " + blockInfo.toString(), e);
       }
     }
+
+    return node2Blocks;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/43e34fce/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java b/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
new file mode 100644
index 0000000..9c66ada
--- /dev/null
+++ b/processing/src/test/java/org/apache/carbondata/processing/util/CarbonLoaderUtilTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.util;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.block.Distributable;
+import org.apache.carbondata.core.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+
+import org.apache.commons.lang3.StringUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CarbonLoaderUtilTest {
+  private final static LogService LOGGER
+      = LogServiceFactory.getLogService(CarbonLoaderUtilTest.class.getName());
+
+  private List<Distributable> generateBlocks() {
+    List<Distributable> blockInfos = new ArrayList<>();
+    String filePath = "/fakepath";
+    String blockId = "1";
+
+    String[] locations = new String[] { "host2", "host3" };
+    ColumnarFormatVersion version = ColumnarFormatVersion.V1;
+
+    TableBlockInfo tableBlockInfo1 = new TableBlockInfo(filePath + "_a", 0,
+        blockId, locations, 30 * 1024 * 1024, version, null);
+    blockInfos.add(tableBlockInfo1);
+
+    TableBlockInfo tableBlockInfo2 = new TableBlockInfo(filePath + "_b", 0,
+        blockId, locations, 40 * 1024 * 1024, version, null);
+    blockInfos.add(tableBlockInfo2);
+
+    TableBlockInfo tableBlockInfo3 = new TableBlockInfo(filePath + "_c", 0,
+        blockId, locations, 20 * 1024 * 1024, version, null);
+    blockInfos.add(tableBlockInfo3);
+
+    TableBlockInfo tableBlockInfo4 = new TableBlockInfo(filePath + "_d", 0,
+        blockId, locations, 1, version, null);
+    blockInfos.add(tableBlockInfo4);
+
+    TableBlockInfo tableBlockInfo5 = new TableBlockInfo(filePath + "_e", 0,
+        blockId, locations, 1, version, null);
+    blockInfos.add(tableBlockInfo5);
+
+    TableBlockInfo tableBlockInfo6 = new TableBlockInfo(filePath + "_f", 0,
+        blockId, locations, 1, version, null);
+    blockInfos.add(tableBlockInfo6);
+
+    TableBlockInfo tableBlockInfo7 = new TableBlockInfo(filePath + "_g", 0,
+        blockId, locations, 1, version, null);
+    blockInfos.add(tableBlockInfo7);
+    return blockInfos;
+  }
+
+  private List<String> generateExecutors() {
+    List<String> activeNodes = new ArrayList<>();
+    activeNodes.add("host1");
+    activeNodes.add("host2");
+    activeNodes.add("host3");
+    return activeNodes;
+  }
+
+  @Test
+  public void testNodeBlockMappingByDataSize() throws Exception {
+    List<Distributable> blockInfos = generateBlocks();
+    List<String> activeNodes = generateExecutors();
+
+    // the blocks are assigned by size, so the number of block for each node are different
+    Map<String, List<Distributable>> nodeMappingBySize =
+        CarbonLoaderUtil.nodeBlockMapping(blockInfos, -1, activeNodes,
+            CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_SIZE_FIRST);
+    LOGGER.info(convertMapListAsString(nodeMappingBySize));
+    Assert.assertEquals(3, nodeMappingBySize.size());
+    for (Map.Entry<String, List<Distributable>> entry : nodeMappingBySize.entrySet()) {
+      if (entry.getValue().size() == 1) {
+        // only contains the biggest block
+        Assert.assertEquals(40 * 1024 * 1024L,
+            ((TableBlockInfo) entry.getValue().get(0)).getBlockLength());
+      } else {
+        Assert.assertTrue(entry.getValue().size() > 1);
+      }
+    }
+
+    // the blocks are assigned by number, so the number of blocks for each node are nearly the same
+    Map<String, List<Distributable>> nodeMappingByNum =
+        CarbonLoaderUtil.nodeBlockMapping(blockInfos, -1, activeNodes,
+            CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_NUM_FIRST);
+    LOGGER.info(convertMapListAsString(nodeMappingByNum));
+    Assert.assertEquals(3, nodeMappingBySize.size());
+    for (Map.Entry<String, List<Distributable>> entry : nodeMappingByNum.entrySet()) {
+      Assert.assertTrue(entry.getValue().size() == blockInfos.size() / 3
+          || entry.getValue().size() == blockInfos.size() / 3 + 1);
+    }
+  }
+
+  private <K, T> String convertMapListAsString(Map<K, List<T>> mapList) {
+    StringBuffer sb = new StringBuffer();
+    for (Map.Entry<K, List<T>> entry : mapList.entrySet()) {
+      String key = entry.getKey().toString();
+      String value = StringUtils.join(entry.getValue(), ", ");
+      sb.append(key).append(" -- ").append(value).append(System.lineSeparator());
+    }
+    return sb.toString();
+  }
+}
\ No newline at end of file


[23/49] carbondata git commit: [CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to executor for s3 implementation in cluster mode.

Posted by ja...@apache.org.
[CARBONDATA-2080] [S3-Implementation] Propagated hadoopConf from driver to executor for s3 implementation in cluster mode.

Problem : hadoopconf was not getting propagated from driver to the executor that's why load was failing to the distributed environment.
Solution: Setting the Hadoop conf in base class CarbonRDD
How to verify this PR :
Execute the load in the cluster mode It should be a success using location s3.

This closes #1860


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

Branch: refs/heads/carbonstore-rebase4
Commit: 46851d8caecf5091b6bd20e3798ae2bb848e5317
Parents: 4d453d4
Author: Jatin <ja...@knoldus.in>
Authored: Thu Jan 25 16:53:00 2018 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 00:21:32 2018 +0800

----------------------------------------------------------------------
 .../spark/rdd/AlterTableAddColumnRDD.scala      |  2 +-
 .../spark/rdd/AlterTableDropColumnRDD.scala     |  2 +-
 .../spark/rdd/CarbonCleanFilesRDD.scala         |  2 +-
 .../spark/rdd/CarbonDeleteLoadByDateRDD.scala   |  2 +-
 .../spark/rdd/CarbonDeleteLoadRDD.scala         |  2 +-
 .../spark/rdd/CarbonDropPartitionRDD.scala      |  2 +-
 .../spark/rdd/CarbonDropTableRDD.scala          |  2 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |  3 +-
 .../spark/rdd/CarbonMergeFilesRDD.scala         |  0
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  2 +-
 .../apache/carbondata/spark/rdd/CarbonRDD.scala | 32 ++++++++++++++++++--
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  2 +-
 .../carbondata/spark/rdd/SparkDataMapJob.scala  |  2 +-
 .../apache/spark/rdd/DataLoadCoalescedRDD.scala |  3 +-
 .../apache/spark/rdd/UpdateCoalescedRDD.scala   |  2 +-
 .../carbondata/streaming/StreamHandoffRDD.scala |  2 +-
 16 files changed, 45 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
index 56a66b9..7c1edea 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableAddColumnRDD.scala
@@ -50,7 +50,7 @@ class AddColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Par
 class AlterTableAddColumnRDD[K, V](sc: SparkContext,
     @transient newColumns: Seq[ColumnSchema],
     identifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
+  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil, sc.hadoopConfiguration) {
 
   val lockType: String = CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_TYPE,
     CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
index 248f351..e14524e 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableDropColumnRDD.scala
@@ -48,7 +48,7 @@ class DropColumnPartition(rddId: Int, idx: Int, schema: ColumnSchema) extends Pa
 class AlterTableDropColumnRDD[K, V](sc: SparkContext,
     @transient newColumns: Seq[ColumnSchema],
     carbonTableIdentifier: AbsoluteTableIdentifier)
-  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil) {
+  extends CarbonRDD[(Int, SegmentStatus)](sc, Nil, sc.hadoopConfiguration) {
 
   override def getPartitions: Array[Partition] = {
     newColumns.zipWithIndex.map { column =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
index 32523d8..9936a2a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
@@ -33,7 +33,7 @@ class CarbonCleanFilesRDD[V: ClassTag](
     databaseName: String,
     tableName: String,
     partitioner: Partitioner)
-  extends CarbonRDD[V](sc, Nil) {
+  extends CarbonRDD[V](sc, Nil, sc.hadoopConfiguration) {
 
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadByDateRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadByDateRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadByDateRDD.scala
index 45271a7..b11dfad 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadByDateRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadByDateRDD.scala
@@ -39,7 +39,7 @@ class CarbonDeleteLoadByDateRDD[K, V](
     dimTableName: String,
     storePath: String,
     loadMetadataDetails: List[LoadMetadataDetails])
-  extends CarbonRDD[(K, V)](sc, Nil) {
+  extends CarbonRDD[(K, V)](sc, Nil, sc.hadoopConfiguration) {
 
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala
index 9a1ef33..759ed42 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala
@@ -34,7 +34,7 @@ class CarbonDeleteLoadRDD[V: ClassTag](
     databaseName: String,
     tableName: String,
     partitioner: Partitioner)
-  extends CarbonRDD[V](sc, Nil) {
+  extends CarbonRDD[V](sc, Nil, sc.hadoopConfiguration) {
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
 
   override def getPartitions: Array[Partition] = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala
index 07a2e57..d56e1c2 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropPartitionRDD.scala
@@ -47,7 +47,7 @@ class CarbonDropPartitionRDD(
     segments: Seq[Segment],
     partitions: util.List[PartitionSpec],
     uniqueId: String)
-  extends CarbonRDD[(String, String)](sc, Nil) {
+  extends CarbonRDD[(String, String)](sc, Nil, sc.hadoopConfiguration) {
 
   override def getPartitions: Array[Partition] = {
     segments.zipWithIndex.map {s =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropTableRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropTableRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropTableRDD.scala
index 652720c..f327d88 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropTableRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDropTableRDD.scala
@@ -30,7 +30,7 @@ class CarbonDropTableRDD[V: ClassTag](
     valueClass: Value[V],
     databaseName: String,
     tableName: String)
-  extends CarbonRDD[V](sc, Nil) {
+  extends CarbonRDD[V](sc, Nil, sc.hadoopConfiguration) {
 
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
index 7acf4e2..cf22b3d 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
@@ -506,7 +506,8 @@ class CarbonColumnDictGenerateRDD(carbonLoadModel: CarbonLoadModel,
     table: CarbonTableIdentifier,
     dimensions: Array[CarbonDimension],
     dictFolderPath: String)
-  extends CarbonRDD[(Int, ColumnDistinctValues)](sparkContext, Nil) {
+  extends CarbonRDD[(Int, ColumnDistinctValues)](sparkContext, Nil,
+    sparkContext.hadoopConfiguration) {
 
   override def getPartitions: Array[Partition] = {
     val primDimensions = dictionaryLoadModel.primDimensions

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergeFilesRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergeFilesRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergeFilesRDD.scala
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 3c70619..3fbed6f 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -64,7 +64,7 @@ class CarbonMergerRDD[K, V](
     carbonLoadModel: CarbonLoadModel,
     carbonMergerMapping: CarbonMergerMapping,
     confExecutorsTemp: String)
-  extends CarbonRDD[(K, V)](sc, Nil) {
+  extends CarbonRDD[(K, V)](sc, Nil, sc.hadoopConfiguration) {
 
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
   sc.setLocalProperty("spark.job.interruptOnCancel", "true")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
index bf46f67..6f248d2 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
@@ -17,20 +17,26 @@
 
 package org.apache.carbondata.spark.rdd
 
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+
 import scala.collection.JavaConverters._
 import scala.reflect.ClassTag
 
+import org.apache.hadoop.conf.Configuration
 import org.apache.spark.{Dependency, OneToOneDependency, Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 
+import org.apache.carbondata.core.datastore.compression.CompressorFactory
 import org.apache.carbondata.core.metadata.schema.table.TableInfo
 import org.apache.carbondata.core.util._
+import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 
 /**
  * This RDD maintains session level ThreadLocal
  */
 abstract class CarbonRDD[T: ClassTag](@transient sc: SparkContext,
-    @transient private var deps: Seq[Dependency[_]]) extends RDD[T](sc, deps) {
+    @transient private var deps: Seq[Dependency[_]],
+    @transient hadoopConf: Configuration) extends RDD[T](sc, deps) {
 
   val carbonSessionInfo: CarbonSessionInfo = {
     var info = ThreadLocalSessionInfo.getCarbonSessionInfo
@@ -42,14 +48,24 @@ abstract class CarbonRDD[T: ClassTag](@transient sc: SparkContext,
     info
   }
 
+  private val confBytes = {
+    val bao = new ByteArrayOutputStream()
+    val oos = new ObjectOutputStream(bao)
+    hadoopConf.write(oos)
+    oos.close()
+    CompressorFactory.getInstance().getCompressor.compressByte(bao.toByteArray)
+  }
+
   /** Construct an RDD with just a one-to-one dependency on one parent */
   def this(@transient oneParent: RDD[_]) =
-    this (oneParent.context, List(new OneToOneDependency(oneParent)))
+    this (oneParent.context, List(new OneToOneDependency(oneParent)),
+      oneParent.sparkContext.hadoopConfiguration)
 
   // RDD compute logic should be here
   def internalCompute(split: Partition, context: TaskContext): Iterator[T]
 
   final def compute(split: Partition, context: TaskContext): Iterator[T] = {
+    CarbonInputFormatUtil.setS3Configurations(getConf)
     ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
     TaskMetricsMap.threadLocal.set(Thread.currentThread().getId)
     val carbonTaskInfo = new CarbonTaskInfo
@@ -59,6 +75,16 @@ abstract class CarbonRDD[T: ClassTag](@transient sc: SparkContext,
       map(f => CarbonProperties.getInstance().addProperty(f._1, f._2))
     internalCompute(split, context)
   }
+
+  private def getConf: Configuration = {
+    val configuration = new Configuration(false)
+    val bai = new ByteArrayInputStream(CompressorFactory.getInstance().getCompressor
+      .unCompressByte(confBytes))
+    val ois = new ObjectInputStream(bai)
+    configuration.readFields(ois)
+    ois.close()
+    configuration
+  }
 }
 
 /**
@@ -67,7 +93,7 @@ abstract class CarbonRDD[T: ClassTag](@transient sc: SparkContext,
 abstract class CarbonRDDWithTableInfo[T: ClassTag](
     @transient sc: SparkContext,
     @transient private var deps: Seq[Dependency[_]],
-    serializedTableInfo: Array[Byte]) extends CarbonRDD[T](sc, deps) {
+    serializedTableInfo: Array[Byte]) extends CarbonRDD[T](sc, deps, sc.hadoopConfiguration) {
 
   def this(@transient oneParent: RDD[_], serializedTableInfo: Array[Byte]) =
     this (oneParent.context, List(new OneToOneDependency(oneParent)), serializedTableInfo)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index e17824f..06acbba 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -182,7 +182,7 @@ class NewCarbonDataLoadRDD[K, V](
     carbonLoadModel: CarbonLoadModel,
     blocksGroupBy: Array[(String, Array[BlockDetails])],
     @transient hadoopConf: Configuration)
-  extends CarbonRDD[(K, V)](sc, Nil) {
+  extends CarbonRDD[(K, V)](sc, Nil, hadoopConf) {
 
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
index 600cd80..60052f0 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/SparkDataMapJob.scala
@@ -58,7 +58,7 @@ class DataMapRDDPartition(rddId: Int, idx: Int, val inputSplit: InputSplit) exte
 class DataMapPruneRDD(sc: SparkContext,
     dataMapFormat: DistributableDataMapFormat,
     resolverIntf: FilterResolverIntf)
-  extends CarbonRDD[(ExtendedBlocklet)](sc, Nil) {
+  extends CarbonRDD[(ExtendedBlocklet)](sc, Nil, sc.hadoopConfiguration) {
 
   private val jobTrackerId: String = {
     val formatter = new SimpleDateFormat("yyyyMMddHHmm")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala b/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala
index 2157799..6a97477 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/rdd/DataLoadCoalescedRDD.scala
@@ -29,7 +29,8 @@ case class DataLoadPartitionWrap[T: ClassTag](rdd: RDD[T], partition: Partition)
 class DataLoadCoalescedRDD[T: ClassTag](
     @transient var prev: RDD[T],
     nodeList: Array[String])
-  extends CarbonRDD[DataLoadPartitionWrap[T]](prev.context, Nil) {
+  extends CarbonRDD[DataLoadPartitionWrap[T]](prev.context, Nil,
+    prev.sparkContext.hadoopConfiguration) {
 
   override def getPartitions: Array[Partition] = {
     new DataLoadPartitionCoalescer(prev, nodeList).run

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/integration/spark-common/src/main/scala/org/apache/spark/rdd/UpdateCoalescedRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/rdd/UpdateCoalescedRDD.scala b/integration/spark-common/src/main/scala/org/apache/spark/rdd/UpdateCoalescedRDD.scala
index 9befcaa..bcca7ed 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/rdd/UpdateCoalescedRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/rdd/UpdateCoalescedRDD.scala
@@ -29,7 +29,7 @@ import org.apache.carbondata.spark.rdd.CarbonRDD
 class UpdateCoalescedRDD[T: ClassTag](
     @transient var prev: RDD[T],
     nodeList: Array[String])
-  extends CarbonRDD[T](prev.context, Nil) {
+  extends CarbonRDD[T](prev.context, Nil, prev.sparkContext.hadoopConfiguration) {
 
   override def getPartitions: Array[Partition] = {
     new DataLoadPartitionCoalescer(prev, nodeList).run

http://git-wip-us.apache.org/repos/asf/carbondata/blob/46851d8c/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
index c9e61d3..7f5093f 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamHandoffRDD.scala
@@ -97,7 +97,7 @@ class StreamHandoffRDD[K, V](
     result: HandoffResult[K, V],
     carbonLoadModel: CarbonLoadModel,
     handOffSegmentId: String
-) extends CarbonRDD[(K, V)](sc, Nil) {
+) extends CarbonRDD[(K, V)](sc, Nil, sc.hadoopConfiguration) {
 
   private val jobTrackerId: String = {
     val formatter = new SimpleDateFormat("yyyyMMddHHmm")


[44/49] carbondata git commit: Support generating assembling JAR for store-sdk module

Posted by ja...@apache.org.
Support generating assembling JAR for store-sdk module

Support generating assembling JAR for store-sdk module and remove junit dependency

This closes #1976


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

Branch: refs/heads/carbonstore-rebase4
Commit: 33a6d2bc31d377a2d1ccbdd422979cd61ac66292
Parents: d6d6f10
Author: Jacky Li <ja...@qq.com>
Authored: Tue Feb 13 09:12:09 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:20:47 2018 +0800

----------------------------------------------------------------------
 common/pom.xml                                |  2 +
 core/pom.xml                                  |  2 +
 hadoop/pom.xml                                |  1 +
 integration/presto/pom.xml                    |  3 +-
 integration/spark-common-cluster-test/pom.xml |  2 +-
 integration/spark-common-test/pom.xml         |  3 +-
 integration/spark-common/pom.xml              |  2 +-
 integration/spark2/pom.xml                    |  2 +-
 pom.xml                                       |  5 +++
 processing/pom.xml                            |  1 +
 store/sdk/pom.xml                             | 50 +++++++++++++++++++++-
 streaming/pom.xml                             |  1 -
 12 files changed, 66 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 5550129..433d575 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -42,10 +42,12 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.jmockit</groupId>
       <artifactId>jmockit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 92c9607..824de0d 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -70,10 +70,12 @@
     <dependency>
       <groupId>org.jmockit</groupId>
       <artifactId>jmockit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.spark</groupId>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop/pom.xml b/hadoop/pom.xml
index 2aaac99..c3964c5 100644
--- a/hadoop/pom.xml
+++ b/hadoop/pom.xml
@@ -42,6 +42,7 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/integration/presto/pom.xml
----------------------------------------------------------------------
diff --git a/integration/presto/pom.xml b/integration/presto/pom.xml
index aaaf175..0abcf38 100644
--- a/integration/presto/pom.xml
+++ b/integration/presto/pom.xml
@@ -193,7 +193,7 @@
         </exclusion>
         <exclusion>
           <groupId>org.scalatest</groupId>
-          <artifactId>scalatest_2.11</artifactId>
+          <artifactId>scalatest_${scala.binary.version}</artifactId>
         </exclusion>
         <exclusion>
           <groupId>org.apache.zookeeper</groupId>
@@ -330,7 +330,6 @@
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
-      <version>2.2.1</version>
       <scope>test</scope>
     </dependency>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/integration/spark-common-cluster-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-cluster-test/pom.xml b/integration/spark-common-cluster-test/pom.xml
index fd907a3..028da11 100644
--- a/integration/spark-common-cluster-test/pom.xml
+++ b/integration/spark-common-cluster-test/pom.xml
@@ -49,11 +49,11 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
-      <version>2.2.1</version>
       <scope>test</scope>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/integration/spark-common-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/pom.xml b/integration/spark-common-test/pom.xml
index 67a2317..d1c04ae 100644
--- a/integration/spark-common-test/pom.xml
+++ b/integration/spark-common-test/pom.xml
@@ -106,16 +106,17 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
-      <version>2.2.1</version>
       <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.jmockit</groupId>
       <artifactId>jmockit</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/integration/spark-common/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common/pom.xml b/integration/spark-common/pom.xml
index 295d62b..16f327d 100644
--- a/integration/spark-common/pom.xml
+++ b/integration/spark-common/pom.xml
@@ -58,11 +58,11 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
-      <version>2.2.1</version>
       <scope>provided</scope>
     </dependency>
   </dependencies>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index aac1ff6..5b2da93 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -50,11 +50,11 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
-      <version>2.2.1</version>
       <scope>test</scope>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 520d770..0891972 100644
--- a/pom.xml
+++ b/pom.xml
@@ -262,6 +262,11 @@
         <version>4.11</version>
         <scope>test</scope>
       </dependency>
+      <dependency>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest_${scala.binary.version}</artifactId>
+        <version>2.2.1</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/processing/pom.xml
----------------------------------------------------------------------
diff --git a/processing/pom.xml b/processing/pom.xml
index acd97fb..466a300 100644
--- a/processing/pom.xml
+++ b/processing/pom.xml
@@ -68,6 +68,7 @@
     <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
+      <scope>test</scope>
     </dependency>
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/store/sdk/pom.xml
----------------------------------------------------------------------
diff --git a/store/sdk/pom.xml b/store/sdk/pom.xml
index 51d2cf9..54fba55 100644
--- a/store/sdk/pom.xml
+++ b/store/sdk/pom.xml
@@ -32,7 +32,6 @@
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
-      <version>2.2.1</version>
       <scope>test</scope>
     </dependency>
   </dependencies>
@@ -125,6 +124,55 @@
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-deploy-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-install-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <configuration>
+          <shadedArtifactAttached>false</shadedArtifactAttached>
+          <promoteTransitiveDependencies>true</promoteTransitiveDependencies>
+          <outputFile>carbondata-sdk.jar</outputFile>
+          <artifactSet>
+            <includes>
+              <include>*:*</include>
+            </includes>
+          </artifactSet>
+          <filters>
+            <filter>
+              <artifact>*:*</artifact>
+              <excludes>
+                <exclude>org/datanucleus/**</exclude>
+                <exclude>META-INF/*.SF</exclude>
+                <exclude>META-INF/*.DSA</exclude>
+                <exclude>META-INF/*.RSA</exclude>
+                <exclude>META-INF/vfs-providers.xml</exclude>
+                <exclude>io/netty/**</exclude>
+              </excludes>
+            </filter>
+          </filters>
+        </configuration>
+        <executions>
+          <execution>
+            <phase>package</phase>
+            <goals>
+              <goal>shade</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
   </build>
 </project>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/33a6d2bc/streaming/pom.xml
----------------------------------------------------------------------
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 987e530..dc66aa8 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -33,7 +33,6 @@
     <dependency>
       <groupId>org.scalatest</groupId>
       <artifactId>scalatest_${scala.binary.version}</artifactId>
-      <version>2.2.1</version>
       <scope>test</scope>
     </dependency>
   </dependencies>


[38/49] carbondata git commit: [CARBONDATA-2159] Remove carbon-spark dependency in store-sdk module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
new file mode 100644
index 0000000..fbb93b6
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -0,0 +1,322 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.model;
+
+import java.io.IOException;
+import java.text.SimpleDateFormat;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.Maps;
+import org.apache.carbondata.common.Strings;
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.constants.LoggerAction;
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
+import org.apache.carbondata.processing.util.TableOptionConstant;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Builder for {@link CarbonLoadModel}
+ */
+@InterfaceAudience.Developer
+public class CarbonLoadModelBuilder {
+
+  private CarbonTable table;
+
+  public CarbonLoadModelBuilder(CarbonTable table) {
+    this.table = table;
+  }
+
+  /**
+   * build CarbonLoadModel for data loading
+   * @param options Load options from user input
+   * @return a new CarbonLoadModel instance
+   */
+  public CarbonLoadModel build(
+      Map<String, String> options) throws InvalidLoadOptionException, IOException {
+    Map<String, String> optionsFinal = LoadOption.fillOptionWithDefaultValue(options);
+    optionsFinal.put("sort_scope", "no_sort");
+    if (!options.containsKey("fileheader")) {
+      List<CarbonColumn> csvHeader = table.getCreateOrderColumn(table.getTableName());
+      String[] columns = new String[csvHeader.size()];
+      for (int i = 0; i < columns.length; i++) {
+        columns[i] = csvHeader.get(i).getColName();
+      }
+      optionsFinal.put("fileheader", Strings.mkString(columns, ","));
+    }
+    CarbonLoadModel model = new CarbonLoadModel();
+
+    // we have provided 'fileheader', so it hadoopConf can be null
+    build(options, optionsFinal, model, null);
+
+    // set default values
+    model.setTimestampformat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+    model.setDateFormat(CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+    model.setUseOnePass(Boolean.parseBoolean(Maps.getOrDefault(options, "onepass", "false")));
+    model.setDictionaryServerHost(Maps.getOrDefault(options, "dicthost", null));
+    try {
+      model.setDictionaryServerPort(Integer.parseInt(Maps.getOrDefault(options, "dictport", "-1")));
+    } catch (NumberFormatException e) {
+      throw new InvalidLoadOptionException(e.getMessage());
+    }
+    return model;
+  }
+
+  /**
+   * build CarbonLoadModel for data loading
+   * @param options Load options from user input
+   * @param optionsFinal Load options that populated with default values for optional options
+   * @param carbonLoadModel The output load model
+   * @param hadoopConf hadoopConf is needed to read CSV header if there 'fileheader' is not set in
+   *                   user provided load options
+   */
+  public void build(
+      Map<String, String> options,
+      Map<String, String> optionsFinal,
+      CarbonLoadModel carbonLoadModel,
+      Configuration hadoopConf) throws InvalidLoadOptionException, IOException {
+    carbonLoadModel.setTableName(table.getTableName());
+    carbonLoadModel.setDatabaseName(table.getDatabaseName());
+    carbonLoadModel.setTablePath(table.getTablePath());
+    carbonLoadModel.setTableName(table.getTableName());
+    CarbonDataLoadSchema dataLoadSchema = new CarbonDataLoadSchema(table);
+    // Need to fill dimension relation
+    carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema);
+    String sort_scope = optionsFinal.get("sort_scope");
+    String single_pass = optionsFinal.get("single_pass");
+    String bad_records_logger_enable = optionsFinal.get("bad_records_logger_enable");
+    String bad_records_action = optionsFinal.get("bad_records_action");
+    String bad_record_path = optionsFinal.get("bad_record_path");
+    String global_sort_partitions = optionsFinal.get("global_sort_partitions");
+    String timestampformat = optionsFinal.get("timestampformat");
+    String dateFormat = optionsFinal.get("dateformat");
+    String delimeter = optionsFinal.get("delimiter");
+    String complex_delimeter_level1 = optionsFinal.get("complex_delimiter_level_1");
+    String complex_delimeter_level2 = optionsFinal.get("complex_delimiter_level_2");
+    String all_dictionary_path = optionsFinal.get("all_dictionary_path");
+    String column_dict = optionsFinal.get("columndict");
+    validateDateTimeFormat(timestampformat, "TimestampFormat");
+    validateDateTimeFormat(dateFormat, "DateFormat");
+    validateSortScope(sort_scope);
+
+    if (Boolean.parseBoolean(bad_records_logger_enable) ||
+        LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
+      bad_record_path = CarbonUtil.checkAndAppendHDFSUrl(bad_record_path);
+      if (!CarbonUtil.isValidBadStorePath(bad_record_path)) {
+        throw new InvalidLoadOptionException("Invalid bad records location.");
+      }
+    }
+    carbonLoadModel.setBadRecordsLocation(bad_record_path);
+
+    validateGlobalSortPartitions(global_sort_partitions);
+    carbonLoadModel.setEscapeChar(checkDefaultValue(optionsFinal.get("escapechar"), "\\"));
+    carbonLoadModel.setQuoteChar(checkDefaultValue(optionsFinal.get("quotechar"), "\""));
+    carbonLoadModel.setCommentChar(checkDefaultValue(optionsFinal.get("commentchar"), "#"));
+
+    // if there isn't file header in csv file and load sql doesn't provide FILEHEADER option,
+    // we should use table schema to generate file header.
+    String fileHeader = optionsFinal.get("fileheader");
+    String headerOption = options.get("header");
+    if (headerOption != null) {
+      if (!headerOption.equalsIgnoreCase("true") &&
+          !headerOption.equalsIgnoreCase("false")) {
+        throw new InvalidLoadOptionException(
+            "'header' option should be either 'true' or 'false'.");
+      }
+      // whether the csv file has file header, the default value is true
+      if (Boolean.valueOf(headerOption)) {
+        if (!StringUtils.isEmpty(fileHeader)) {
+          throw new InvalidLoadOptionException(
+              "When 'header' option is true, 'fileheader' option is not required.");
+        }
+      } else {
+        if (StringUtils.isEmpty(fileHeader)) {
+          List<CarbonColumn> columns = table.getCreateOrderColumn(table.getTableName());
+          String[] columnNames = new String[columns.size()];
+          for (int i = 0; i < columnNames.length; i++) {
+            columnNames[i] = columns.get(i).getColName();
+          }
+          fileHeader = Strings.mkString(columnNames, ",");
+        }
+      }
+    }
+
+    carbonLoadModel.setTimestampformat(timestampformat);
+    carbonLoadModel.setDateFormat(dateFormat);
+    carbonLoadModel.setDefaultTimestampFormat(
+        CarbonProperties.getInstance().getProperty(
+            CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+
+    carbonLoadModel.setDefaultDateFormat(
+        CarbonProperties.getInstance().getProperty(
+            CarbonCommonConstants.CARBON_DATE_FORMAT,
+            CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
+
+    carbonLoadModel.setSerializationNullFormat(
+        TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," +
+            optionsFinal.get("serialization_null_format"));
+
+    carbonLoadModel.setBadRecordsLoggerEnable(
+        TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + bad_records_logger_enable);
+
+    carbonLoadModel.setBadRecordsAction(
+        TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + bad_records_action.toUpperCase());
+
+    carbonLoadModel.setIsEmptyDataBadRecord(
+        DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," +
+            optionsFinal.get("is_empty_data_bad_record"));
+
+    carbonLoadModel.setSkipEmptyLine(optionsFinal.get("skip_empty_line"));
+
+    carbonLoadModel.setSortScope(sort_scope);
+    carbonLoadModel.setBatchSortSizeInMb(optionsFinal.get("batch_sort_size_inmb"));
+    carbonLoadModel.setGlobalSortPartitions(global_sort_partitions);
+    carbonLoadModel.setUseOnePass(Boolean.parseBoolean(single_pass));
+
+    if (delimeter.equalsIgnoreCase(complex_delimeter_level1) ||
+        complex_delimeter_level1.equalsIgnoreCase(complex_delimeter_level2) ||
+        delimeter.equalsIgnoreCase(complex_delimeter_level2)) {
+      throw new InvalidLoadOptionException("Field Delimiter and Complex types delimiter are same");
+    } else {
+      carbonLoadModel.setComplexDelimiterLevel1(
+          CarbonUtil.delimiterConverter(complex_delimeter_level1));
+      carbonLoadModel.setComplexDelimiterLevel2(
+          CarbonUtil.delimiterConverter(complex_delimeter_level2));
+    }
+    // set local dictionary path, and dictionary file extension
+    carbonLoadModel.setAllDictPath(all_dictionary_path);
+    carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimeter));
+    carbonLoadModel.setCsvHeader(fileHeader);
+    carbonLoadModel.setColDictFilePath(column_dict);
+    carbonLoadModel.setCsvHeaderColumns(
+        LoadOption.getCsvHeaderColumns(carbonLoadModel, hadoopConf));
+
+    int validatedMaxColumns = validateMaxColumns(
+        carbonLoadModel.getCsvHeaderColumns(),
+        optionsFinal.get("maxcolumns"));
+
+    carbonLoadModel.setMaxColumns(String.valueOf(validatedMaxColumns));
+    carbonLoadModel.readAndSetLoadMetadataDetails();
+  }
+
+  private int validateMaxColumns(String[] csvHeaders, String maxColumns)
+      throws InvalidLoadOptionException {
+    /*
+    User configures both csvheadercolumns, maxcolumns,
+      if csvheadercolumns >= maxcolumns, give error
+      if maxcolumns > threashold, give error
+    User configures csvheadercolumns
+      if csvheadercolumns >= maxcolumns(default) then maxcolumns = csvheadercolumns+1
+      if csvheadercolumns >= threashold, give error
+    User configures nothing
+      if csvheadercolumns >= maxcolumns(default) then maxcolumns = csvheadercolumns+1
+      if csvheadercolumns >= threashold, give error
+     */
+    int columnCountInSchema = csvHeaders.length;
+    int maxNumberOfColumnsForParsing = 0;
+    Integer maxColumnsInt = getMaxColumnValue(maxColumns);
+    if (maxColumnsInt != null) {
+      if (columnCountInSchema >= maxColumnsInt) {
+        throw new InvalidLoadOptionException(
+            "csv headers should be less than the max columns " + maxColumnsInt);
+      } else if (maxColumnsInt > CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) {
+        throw new InvalidLoadOptionException(
+            "max columns cannot be greater than the threshold value: " +
+                CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING);
+      } else {
+        maxNumberOfColumnsForParsing = maxColumnsInt;
+      }
+    } else if (columnCountInSchema >= CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) {
+      throw new InvalidLoadOptionException(
+          "csv header columns should be less than max threashold: " +
+              CSVInputFormat.THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING);
+    } else if (columnCountInSchema >= CSVInputFormat.DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) {
+      maxNumberOfColumnsForParsing = columnCountInSchema + 1;
+    } else {
+      maxNumberOfColumnsForParsing = CSVInputFormat.DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING;
+    }
+    return maxNumberOfColumnsForParsing;
+  }
+
+  private Integer getMaxColumnValue(String maxColumn) {
+    return (maxColumn == null) ? null : Integer.parseInt(maxColumn);
+  }
+
+  /**
+   * validates both timestamp and date for illegal values
+   */
+  private void validateDateTimeFormat(String dateTimeLoadFormat, String dateTimeLoadOption)
+      throws InvalidLoadOptionException {
+    // allowing empty value to be configured for dateformat option.
+    if (dateTimeLoadFormat != null && !dateTimeLoadFormat.trim().equalsIgnoreCase("")) {
+      try {
+        new SimpleDateFormat(dateTimeLoadFormat);
+      } catch (IllegalArgumentException e) {
+        throw new InvalidLoadOptionException(
+            "Error: Wrong option: " + dateTimeLoadFormat + " is provided for option "
+                + dateTimeLoadOption);
+      }
+    }
+  }
+
+  private void validateSortScope(String sortScope) throws InvalidLoadOptionException {
+    if (sortScope != null) {
+      // Don't support use global sort on partitioned table.
+      if (table.getPartitionInfo(table.getTableName()) != null &&
+          sortScope.equalsIgnoreCase(SortScopeOptions.SortScope.GLOBAL_SORT.toString())) {
+        throw new InvalidLoadOptionException("Don't support use global sort on partitioned table.");
+      }
+    }
+  }
+
+  private void validateGlobalSortPartitions(String globalSortPartitions)
+      throws InvalidLoadOptionException {
+    if (globalSortPartitions != null) {
+      try {
+        int num = Integer.parseInt(globalSortPartitions);
+        if (num <= 0) {
+          throw new InvalidLoadOptionException("'GLOBAL_SORT_PARTITIONS' should be greater than 0");
+        }
+      } catch (NumberFormatException e) {
+        throw new InvalidLoadOptionException(e.getMessage());
+      }
+    }
+  }
+
+  /**
+   * check whether using default value or not
+   */
+  private String checkDefaultValue(String value, String defaultValue) {
+    if (StringUtils.isEmpty(value)) {
+      return defaultValue;
+    } else {
+      return value;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
new file mode 100644
index 0000000..8ec93a9
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/LoadOption.java
@@ -0,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.apache.carbondata.processing.loading.model;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.carbondata.common.Maps;
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+import org.apache.carbondata.processing.util.CarbonLoaderUtil;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Provide utilities to populate loading options
+ */
+@InterfaceAudience.Developer
+public class LoadOption {
+
+  private static LogService LOG = LogServiceFactory.getLogService(LoadOption.class.getName());
+
+  /**
+   * Based on the input options, fill and return data loading options with default value
+   */
+  public static Map<String, String> fillOptionWithDefaultValue(
+      Map<String, String> options) throws InvalidLoadOptionException {
+    Map<String, String> optionsFinal = new HashMap<>();
+    optionsFinal.put("delimiter", Maps.getOrDefault(options, "delimiter", ","));
+    optionsFinal.put("quotechar", Maps.getOrDefault(options, "quotechar", "\""));
+    optionsFinal.put("fileheader", Maps.getOrDefault(options, "fileheader", ""));
+    optionsFinal.put("commentchar", Maps.getOrDefault(options, "commentchar", "#"));
+    optionsFinal.put("columndict", Maps.getOrDefault(options, "columndict", null));
+
+    optionsFinal.put(
+        "escapechar",
+        CarbonLoaderUtil.getEscapeChar(Maps.getOrDefault(options,"escapechar", "\\")));
+
+    optionsFinal.put(
+        "serialization_null_format",
+        Maps.getOrDefault(options, "serialization_null_format", "\\N"));
+
+    optionsFinal.put(
+        "bad_records_logger_enable",
+        Maps.getOrDefault(
+            options,
+            "bad_records_logger_enable",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE,
+                CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT)));
+
+    String badRecordActionValue = CarbonProperties.getInstance().getProperty(
+        CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION,
+        CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION_DEFAULT);
+
+    optionsFinal.put(
+        "bad_records_action",
+        Maps.getOrDefault(
+            options,
+            "bad_records_action",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION,
+                badRecordActionValue)));
+
+    optionsFinal.put(
+        "is_empty_data_bad_record",
+        Maps.getOrDefault(
+            options,
+            "is_empty_data_bad_record",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD,
+                CarbonLoadOptionConstants.CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT)));
+
+    optionsFinal.put(
+        "skip_empty_line",
+        Maps.getOrDefault(
+            options,
+            "skip_empty_line",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_SKIP_EMPTY_LINE)));
+
+    optionsFinal.put(
+        "all_dictionary_path",
+        Maps.getOrDefault(options, "all_dictionary_path", ""));
+
+    optionsFinal.put(
+        "complex_delimiter_level_1",
+        Maps.getOrDefault(options,"complex_delimiter_level_1", "\\$"));
+
+    optionsFinal.put(
+        "complex_delimiter_level_2",
+        Maps.getOrDefault(options, "complex_delimiter_level_2", "\\:"));
+
+    optionsFinal.put(
+        "dateformat",
+        Maps.getOrDefault(
+            options,
+            "dateformat",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT,
+                CarbonLoadOptionConstants.CARBON_OPTIONS_DATEFORMAT_DEFAULT)));
+
+    optionsFinal.put(
+        "timestampformat",
+        Maps.getOrDefault(
+            options,
+            "timestampformat",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_TIMESTAMPFORMAT,
+                CarbonLoadOptionConstants.CARBON_OPTIONS_TIMESTAMPFORMAT_DEFAULT)));
+
+    optionsFinal.put(
+        "global_sort_partitions",
+        Maps.getOrDefault(
+            options,
+            "global_sort_partitions",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS,
+                null)));
+
+    optionsFinal.put("maxcolumns", Maps.getOrDefault(options, "maxcolumns", null));
+
+    optionsFinal.put(
+        "batch_sort_size_inmb",
+        Maps.getOrDefault(
+            options,
+            "batch_sort_size_inmb",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_BATCH_SORT_SIZE_INMB,
+                CarbonProperties.getInstance().getProperty(
+                    CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
+                    CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB_DEFAULT))));
+
+    optionsFinal.put(
+        "bad_record_path",
+        Maps.getOrDefault(
+            options,
+            "bad_record_path",
+            CarbonProperties.getInstance().getProperty(
+                CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
+                CarbonProperties.getInstance().getProperty(
+                    CarbonCommonConstants.CARBON_BADRECORDS_LOC,
+                    CarbonCommonConstants.CARBON_BADRECORDS_LOC_DEFAULT_VAL))));
+
+    String useOnePass = Maps.getOrDefault(
+        options,
+        "single_pass",
+        CarbonProperties.getInstance().getProperty(
+            CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS,
+            CarbonLoadOptionConstants.CARBON_OPTIONS_SINGLE_PASS_DEFAULT)).trim().toLowerCase();
+
+    boolean singlePass;
+
+    if (useOnePass.equalsIgnoreCase("true")) {
+      singlePass = true;
+    } else {
+      // when single_pass = false  and if either alldictionarypath
+      // or columnDict is configured the do not allow load
+      if (StringUtils.isNotEmpty(optionsFinal.get("all_dictionary_path")) ||
+          StringUtils.isNotEmpty(optionsFinal.get("columndict"))) {
+        throw new InvalidLoadOptionException(
+            "Can not use all_dictionary_path or columndict without single_pass.");
+      } else {
+        singlePass = false;
+      }
+    }
+
+    optionsFinal.put("single_pass", String.valueOf(singlePass));
+    return optionsFinal;
+  }
+
+  /**
+   * Return CSV header field names
+   */
+  public static String[] getCsvHeaderColumns(
+      CarbonLoadModel carbonLoadModel,
+      Configuration hadoopConf) throws IOException {
+    String delimiter;
+    if (StringUtils.isEmpty(carbonLoadModel.getCsvDelimiter())) {
+      delimiter = CarbonCommonConstants.COMMA;
+    } else {
+      delimiter = CarbonUtil.delimiterConverter(carbonLoadModel.getCsvDelimiter());
+    }
+    String csvFile = null;
+    String csvHeader = carbonLoadModel.getCsvHeader();
+    String[] csvColumns;
+    if (StringUtils.isBlank(csvHeader)) {
+      // read header from csv file
+      csvFile = carbonLoadModel.getFactFilePath().split(",")[0];
+      csvHeader = CarbonUtil.readHeader(csvFile, hadoopConf);
+      if (StringUtils.isBlank(csvHeader)) {
+        throw new CarbonDataLoadingException("First line of the csv is not valid.");
+      }
+      String[] headers = csvHeader.toLowerCase().split(delimiter);
+      csvColumns = new String[headers.length];
+      for (int i = 0; i < csvColumns.length; i++) {
+        csvColumns[i] = headers[i].replaceAll("\"", "").trim();
+      }
+    } else {
+      String[] headers = csvHeader.toLowerCase().split(CarbonCommonConstants.COMMA);
+      csvColumns = new String[headers.length];
+      for (int i = 0; i < csvColumns.length; i++) {
+        csvColumns[i] = headers[i].trim();
+      }
+    }
+
+    if (!CarbonDataProcessorUtil.isHeaderValid(carbonLoadModel.getTableName(), csvColumns,
+        carbonLoadModel.getCarbonDataLoadSchema())) {
+      if (csvFile == null) {
+        LOG.error("CSV header in DDL is not proper."
+            + " Column names in schema and CSV header are not the same.");
+        throw new CarbonDataLoadingException(
+            "CSV header in DDL is not proper. Column names in schema and CSV header are "
+                + "not the same.");
+      } else {
+        LOG.error(
+            "CSV header in input file is not proper. Column names in schema and csv header are not "
+                + "the same. Input file : " + CarbonUtil.removeAKSK(csvFile));
+        throw new CarbonDataLoadingException(
+            "CSV header in input file is not proper. Column names in schema and csv header are not "
+                + "the same. Input file : " + CarbonUtil.removeAKSK(csvFile));
+      }
+    }
+    return csvColumns;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index e9bd3b8..6876355 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -16,14 +16,9 @@
  */
 package org.apache.carbondata.processing.util;
 
-import java.io.BufferedWriter;
-import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.OutputStreamWriter;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.nio.charset.Charset;
-import java.text.SimpleDateFormat;
 import java.util.*;
 
 import org.apache.carbondata.common.logging.LogService;
@@ -40,9 +35,6 @@ import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
 import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.impl.FileFactory.FileType;
-import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
-import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
-import org.apache.carbondata.core.fileoperations.FileWriteOperation;
 import org.apache.carbondata.core.locks.CarbonLockUtil;
 import org.apache.carbondata.core.locks.ICarbonLock;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
@@ -58,9 +50,8 @@ import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.merger.NodeBlockRelation;
 import org.apache.carbondata.processing.merger.NodeMultiBlockRelation;
-import static org.apache.carbondata.core.enums.EscapeSequences.*;
 
-import com.google.gson.Gson;
+import static org.apache.carbondata.core.enums.EscapeSequences.*;
 
 
 public final class CarbonLoaderUtil {
@@ -344,48 +335,6 @@ public final class CarbonLoaderUtil {
     loadMetadataDetails.setLoadStartTime(loadStartTime);
   }
 
-  public static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
-      List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
-    String dataLoadLocation = CarbonTablePath.getTableStatusFilePath(identifier.getTablePath());
-
-    DataOutputStream dataOutputStream;
-    Gson gsonObjectToWrite = new Gson();
-    BufferedWriter brWriter = null;
-
-    AtomicFileOperations writeOperation =
-        new AtomicFileOperationsImpl(dataLoadLocation, FileFactory.getFileType(dataLoadLocation));
-
-    try {
-      dataOutputStream = writeOperation.openForWrite(FileWriteOperation.OVERWRITE);
-      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
-              Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-
-      String metadataInstance = gsonObjectToWrite.toJson(listOfLoadFolderDetails.toArray());
-      brWriter.write(metadataInstance);
-    } finally {
-      try {
-        if (null != brWriter) {
-          brWriter.flush();
-        }
-      } catch (Exception e) {
-        LOGGER.error("error in  flushing ");
-
-      }
-      CarbonUtil.closeStreams(brWriter);
-      writeOperation.close();
-    }
-
-  }
-
-  public static String readCurrentTime() {
-    SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
-    String date = null;
-
-    date = sdf.format(new Date());
-
-    return date;
-  }
-
   public static boolean isValidEscapeSequence(String escapeChar) {
     return escapeChar.equalsIgnoreCase(NEW_LINE.getName()) ||
         escapeChar.equalsIgnoreCase(CARRIAGE_RETURN.getName()) ||
@@ -514,17 +463,6 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * This method will divide the blocks among the nodes as per the data locality
-   *
-   * @param blockInfos
-   * @return
-   */
-  public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos) {
-    // -1 if number of nodes has to be decided based on block location information
-    return nodeBlockMapping(blockInfos, -1);
-  }
-
-  /**
    * the method returns the number of required executors
    *
    * @param blockInfos
@@ -899,25 +837,6 @@ public final class CarbonLoaderUtil {
     CarbonUtil.checkAndCreateFolder(segmentFolder);
   }
 
-  /**
-   * This will update the old table status details before clean files to the latest table status.
-   * @param oldList
-   * @param newList
-   * @return
-   */
-  public static List<LoadMetadataDetails> updateLoadMetadataFromOldToNew(
-      LoadMetadataDetails[] oldList, LoadMetadataDetails[] newList) {
-
-    List<LoadMetadataDetails> newListMetadata =
-        new ArrayList<LoadMetadataDetails>(Arrays.asList(newList));
-    for (LoadMetadataDetails oldSegment : oldList) {
-      if ("false".equalsIgnoreCase(oldSegment.getVisibility())) {
-        newListMetadata.get(newListMetadata.indexOf(oldSegment)).setVisibility("false");
-      }
-    }
-    return newListMetadata;
-  }
-
   /*
    * This method will add data size and index size into tablestatus for each segment. And also
    * returns the size of the segment.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
deleted file mode 100644
index c00cc86..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
+++ /dev/null
@@ -1,210 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.util;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.indexstore.PartitionSpec;
-import org.apache.carbondata.core.locks.CarbonLockFactory;
-import org.apache.carbondata.core.locks.ICarbonLock;
-import org.apache.carbondata.core.locks.LockUsage;
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.core.metadata.SegmentFileStore;
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.core.statusmanager.SegmentStatus;
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-public final class DeleteLoadFolders {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DeleteLoadFolders.class.getName());
-
-  private DeleteLoadFolders() {
-
-  }
-
-  /**
-   * returns segment path
-   *
-   * @param identifier
-   * @param oneLoad
-   * @return
-   */
-  private static String getSegmentPath(AbsoluteTableIdentifier identifier,
-      LoadMetadataDetails oneLoad) {
-    String segmentId = oneLoad.getLoadName();
-    return CarbonTablePath.getSegmentPath(identifier.getTablePath(), segmentId);
-  }
-
-  public static void physicalFactAndMeasureMetadataDeletion(
-      AbsoluteTableIdentifier absoluteTableIdentifier, String metadataPath, boolean isForceDelete,
-      List<PartitionSpec> specs) {
-    LoadMetadataDetails[] currentDetails = SegmentStatusManager.readLoadMetadata(metadataPath);
-    for (LoadMetadataDetails oneLoad : currentDetails) {
-      if (checkIfLoadCanBeDeletedPhysically(oneLoad, isForceDelete)) {
-        try {
-          if (oneLoad.getSegmentFile() != null) {
-            SegmentFileStore
-                .deleteSegment(absoluteTableIdentifier.getTablePath(), oneLoad.getSegmentFile(),
-                    specs);
-          } else {
-            String path = getSegmentPath(absoluteTableIdentifier, oneLoad);
-            boolean status = false;
-            if (FileFactory.isFileExist(path, FileFactory.getFileType(path))) {
-              CarbonFile file = FileFactory.getCarbonFile(path, FileFactory.getFileType(path));
-              CarbonFile[] filesToBeDeleted = file.listFiles(new CarbonFileFilter() {
-
-                @Override public boolean accept(CarbonFile file) {
-                  return (CarbonTablePath.isCarbonDataFile(file.getName()) ||
-                      CarbonTablePath.isCarbonIndexFile(file.getName()));
-                }
-              });
-
-              //if there are no fact and msr metadata files present then no need to keep
-              //entry in metadata.
-              if (filesToBeDeleted.length == 0) {
-                status = true;
-              } else {
-
-                for (CarbonFile eachFile : filesToBeDeleted) {
-                  if (!eachFile.delete()) {
-                    LOGGER.warn("Unable to delete the file as per delete command " + eachFile
-                        .getAbsolutePath());
-                    status = false;
-                  } else {
-                    status = true;
-                  }
-                }
-              }
-              // need to delete the complete folder.
-              if (status) {
-                if (!file.delete()) {
-                  LOGGER.warn("Unable to delete the folder as per delete command " + file
-                      .getAbsolutePath());
-                }
-              }
-
-            } else {
-              LOGGER.warn("Files are not found in segment " + path
-                  + " it seems, files are already being deleted");
-            }
-
-          }
-        } catch (IOException e) {
-          LOGGER.warn("Unable to delete the file as per delete command " + oneLoad.getLoadName());
-        }
-      }
-    }
-  }
-
-  private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrdeletionTimesStamp();
-
-      return CarbonUpdateUtil.isMaxQueryTimeoutExceeded(deletionTime);
-
-    }
-
-    return false;
-  }
-
-  private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus())) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrdeletionTimesStamp();
-
-      return CarbonUpdateUtil.isMaxQueryTimeoutExceeded(deletionTime);
-
-    }
-
-    return false;
-  }
-
-  private static LoadMetadataDetails getCurrentLoadStatusOfSegment(String segmentId,
-      String metadataPath) {
-    LoadMetadataDetails[] currentDetails = SegmentStatusManager.readLoadMetadata(metadataPath);
-    for (LoadMetadataDetails oneLoad : currentDetails) {
-      if (oneLoad.getLoadName().equalsIgnoreCase(segmentId)) {
-        return oneLoad;
-      }
-    }
-    return null;
-  }
-
-  public static boolean deleteLoadFoldersFromFileSystem(
-      AbsoluteTableIdentifier absoluteTableIdentifier, boolean isForceDelete,
-      LoadMetadataDetails[] details, String metadataPath) {
-    boolean isDeleted = false;
-    if (details != null && details.length != 0) {
-      for (LoadMetadataDetails oneLoad : details) {
-        if (checkIfLoadCanBeDeleted(oneLoad, isForceDelete)) {
-          ICarbonLock segmentLock = CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier,
-              CarbonTablePath.addSegmentPrefix(oneLoad.getLoadName()) + LockUsage.LOCK);
-          try {
-            if (oneLoad.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS
-                || oneLoad.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS) {
-              if (segmentLock.lockWithRetries(1, 5)) {
-                LOGGER.info("Info: Acquired segment lock on segment:" + oneLoad.getLoadName());
-                LoadMetadataDetails currentDetails =
-                    getCurrentLoadStatusOfSegment(oneLoad.getLoadName(), metadataPath);
-                if (currentDetails != null && checkIfLoadCanBeDeleted(currentDetails,
-                    isForceDelete)) {
-                  oneLoad.setVisibility("false");
-                  isDeleted = true;
-                  LOGGER.info("Info: Deleted the load " + oneLoad.getLoadName());
-                }
-              } else {
-                LOGGER.info("Info: Load in progress for segment" + oneLoad.getLoadName());
-                return isDeleted;
-              }
-            } else {
-              oneLoad.setVisibility("false");
-              isDeleted = true;
-              LOGGER.info("Info: Deleted the load " + oneLoad.getLoadName());
-            }
-          } finally {
-            segmentLock.unlock();
-            LOGGER.info("Info: Segment lock on segment:" + oneLoad.getLoadName() + " is released");
-          }
-        }
-      }
-    }
-    return isDeleted;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/store/sdk/pom.xml
----------------------------------------------------------------------
diff --git a/store/sdk/pom.xml b/store/sdk/pom.xml
index 6663683..51d2cf9 100644
--- a/store/sdk/pom.xml
+++ b/store/sdk/pom.xml
@@ -21,7 +21,7 @@
   <dependencies>
     <dependency>
       <groupId>org.apache.carbondata</groupId>
-      <artifactId>carbondata-spark-common</artifactId>
+      <artifactId>carbondata-hadoop</artifactId>
       <version>${project.version}</version>
     </dependency>
     <dependency>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
index 51ca09c..e06200a 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
@@ -28,6 +28,7 @@ import java.util.Objects;
 import org.apache.carbondata.common.Strings;
 import org.apache.carbondata.common.annotations.InterfaceAudience;
 import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
@@ -40,7 +41,7 @@ import org.apache.carbondata.core.metadata.schema.table.TableSchemaBuilder;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.ThriftWriter;
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
-import org.apache.carbondata.spark.util.DataLoadingUtil;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModelBuilder;
 
 /**
  * Biulder for {@link CarbonWriter}
@@ -94,9 +95,9 @@ public class CarbonWriterBuilder {
   }
 
   /**
-   * Build a {@link CSVCarbonWriter}, which accepts row in CSV format
+   * Build a {@link CarbonWriter}, which accepts row in CSV format
    */
-  public CarbonWriter buildWriterForCSVInput() throws IOException {
+  public CarbonWriter buildWriterForCSVInput() throws IOException, InvalidLoadOptionException {
     Objects.requireNonNull(schema, "schema should not be null");
     Objects.requireNonNull(path, "path should not be null");
 
@@ -113,7 +114,7 @@ public class CarbonWriterBuilder {
   }
 
   /**
-   * Build a {@link AvroCarbonWriter}, which accepts Avro object
+   * Build a {@link CarbonWriter}, which accepts Avro object
    * @return
    * @throws IOException
    */
@@ -184,11 +185,13 @@ public class CarbonWriterBuilder {
   /**
    * Build a {@link CarbonLoadModel}
    */
-  private CarbonLoadModel buildLoadModel(CarbonTable table) {
+  private CarbonLoadModel buildLoadModel(CarbonTable table)
+      throws InvalidLoadOptionException, IOException {
     Map<String, String> options = new HashMap<>();
     if (sortColumns != null) {
       options.put("sort_columns", Strings.mkString(sortColumns, ","));
     }
-    return DataLoadingUtil.buildCarbonLoadModelJava(table, options);
+    CarbonLoadModelBuilder builder = new CarbonLoadModelBuilder(table);
+    return builder.build(options);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
index 531ec7c..aca2b2d 100644
--- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
@@ -77,7 +77,7 @@ public class CSVCarbonWriterSuite {
         writer.write(new String[]{"robot" + i, String.valueOf(i), String.valueOf((double) i / 2)});
       }
       writer.close();
-    } catch (IOException e) {
+    } catch (Exception e) {
       e.printStackTrace();
       Assert.fail(e.getMessage());
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/06a71586/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
----------------------------------------------------------------------
diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
index 6316d84..bc7b042 100644
--- a/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ b/streaming/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -34,10 +34,9 @@ import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePostExecutionEvent, LoadTablePreExecutionEvent}
-import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.model.{CarbonLoadModel, CarbonLoadModelBuilder, LoadOption}
 import org.apache.carbondata.spark.dictionary.provider.SecureDictionaryServiceProvider
 import org.apache.carbondata.spark.dictionary.server.SecureDictionaryServer
-import org.apache.carbondata.spark.util.DataLoadingUtil
 import org.apache.carbondata.streaming.segment.StreamSegment
 
 /**
@@ -209,17 +208,15 @@ object StreamSinkFactory {
       segmentId: String): CarbonLoadModel = {
     val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
     carbonProperty.addProperty("zookeeper.enable.lock", "false")
-    val optionsFinal = DataLoadingUtil.getDataLoadingOptions(carbonProperty, parameters)
+    val optionsFinal = LoadOption.fillOptionWithDefaultValue(parameters.asJava)
     optionsFinal.put("sort_scope", "no_sort")
     if (parameters.get("fileheader").isEmpty) {
       optionsFinal.put("fileheader", carbonTable.getCreateOrderColumn(carbonTable.getTableName)
         .asScala.map(_.getColName).mkString(","))
     }
     val carbonLoadModel = new CarbonLoadModel()
-    DataLoadingUtil.buildCarbonLoadModel(
-      carbonTable,
-      carbonProperty,
-      parameters,
+    new CarbonLoadModelBuilder(carbonTable).build(
+      parameters.asJava,
       optionsFinal,
       carbonLoadModel,
       hadoopConf)


[37/49] carbondata git commit: [CARBONDATA-1997] Add CarbonWriter SDK API

Posted by ja...@apache.org.
[CARBONDATA-1997] Add CarbonWriter SDK API

Added a new module called store-sdk, and added a CarbonWriter API, it can be used to write Carbondata files to a specified folder, without Spark and Hadoop dependency. User can use this API in any environment.

This closes #1967


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

Branch: refs/heads/carbonstore-rebase4
Commit: 599b9ae975488dfffa170f64a75ef9bd29a22872
Parents: 9713929
Author: Jacky Li <ja...@qq.com>
Authored: Sat Feb 10 19:44:23 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 09:09:16 2018 +0800

----------------------------------------------------------------------
 .../org/apache/carbondata/common/Strings.java   |  40 ++++
 .../apache/carbondata/common/StringsSuite.java  |  53 +++++
 .../core/metadata/schema/table/CarbonTable.java |   7 +
 .../schema/table/CarbonTableBuilder.java        |  72 +++++++
 .../core/metadata/schema/table/TableSchema.java |   7 +
 .../schema/table/TableSchemaBuilder.java        | 107 ++++++++++
 .../schema/table/CarbonTableBuilderSuite.java   |  86 ++++++++
 .../metadata/schema/table/CarbonTableTest.java  |  12 +-
 .../schema/table/TableSchemaBuilderSuite.java   |  56 ++++++
 .../carbondata/spark/util/DataLoadingUtil.scala |  45 +++++
 pom.xml                                         |   7 +
 store/sdk/pom.xml                               | 130 +++++++++++++
 .../carbondata/sdk/file/CSVCarbonWriter.java    |  89 +++++++++
 .../carbondata/sdk/file/CarbonWriter.java       |  51 +++++
 .../sdk/file/CarbonWriterBuilder.java           | 194 +++++++++++++++++++
 .../org/apache/carbondata/sdk/file/Field.java   |  74 +++++++
 .../org/apache/carbondata/sdk/file/Schema.java  |  74 +++++++
 .../sdk/file/CSVCarbonWriterSuite.java          | 127 ++++++++++++
 18 files changed, 1225 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/common/src/main/java/org/apache/carbondata/common/Strings.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/carbondata/common/Strings.java b/common/src/main/java/org/apache/carbondata/common/Strings.java
new file mode 100644
index 0000000..23288dd
--- /dev/null
+++ b/common/src/main/java/org/apache/carbondata/common/Strings.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common;
+
+import java.util.Objects;
+
+public class Strings {
+
+  /**
+   * Provide same function as mkString in Scala.
+   * This is added to avoid JDK 8 dependency.
+   */
+  public static String mkString(String[] strings, String delimeter) {
+    Objects.requireNonNull(strings);
+    Objects.requireNonNull(delimeter);
+    StringBuilder builder = new StringBuilder();
+    for (int i = 0; i < strings.length; i++) {
+      builder.append(strings[i]);
+      if (i != strings.length - 1) {
+        builder.append(delimeter);
+      }
+    }
+    return builder.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/common/src/test/java/org/apache/carbondata/common/StringsSuite.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/carbondata/common/StringsSuite.java b/common/src/test/java/org/apache/carbondata/common/StringsSuite.java
new file mode 100644
index 0000000..65da32b
--- /dev/null
+++ b/common/src/test/java/org/apache/carbondata/common/StringsSuite.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.common;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class StringsSuite {
+
+  @Test(expected = NullPointerException.class)
+  public void testMkStringNullString() {
+    Strings.mkString(null, ",");
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testMkStringNullDelimeter() {
+    Strings.mkString(new String[]{"abc"}, null);
+  }
+  
+  @Test
+  public void testMkString() {
+    String[] strings = new String[]{};
+    String output = Strings.mkString(strings, ",");
+    Assert.assertTrue(output.length() == 0);
+
+    strings = new String[]{"abc"};
+    output = Strings.mkString(strings, ",");
+    Assert.assertEquals("abc", output);
+
+    strings = new String[]{"abc", "def"};
+    output = Strings.mkString(strings, ",");
+    Assert.assertEquals("abc,def", output);
+
+    strings = new String[]{"abc", "def", "ghj"};
+    output = Strings.mkString(strings, ",");
+    Assert.assertEquals("abc,def,ghj", output);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 6ed1197..4d54ad5 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -875,4 +875,11 @@ public class CarbonTable implements Serializable {
 
     return projection;
   }
+
+  /**
+   * Create a {@link CarbonTableBuilder} to create {@link CarbonTable}
+   */
+  public static CarbonTableBuilder builder() {
+    return new CarbonTableBuilder();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java
new file mode 100644
index 0000000..27808f8
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilder.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.metadata.schema.table;
+
+import java.util.ArrayList;
+import java.util.Objects;
+
+/**
+ * Builder for {@link CarbonTable}
+ */
+public class CarbonTableBuilder {
+
+  private String tableName;
+  private String databaseName;
+  private String tablePath;
+  private TableSchema tableSchema;
+
+  public CarbonTableBuilder tableName(String tableName) {
+    Objects.requireNonNull(tableName, "tableName should not be null");
+    this.tableName = tableName;
+    return this;
+  }
+
+  public CarbonTableBuilder databaseName(String databaseName) {
+    Objects.requireNonNull(databaseName, "databaseName should not be null");
+    this.databaseName = databaseName;
+    return this;
+  }
+
+  public CarbonTableBuilder tablePath(String tablePath) {
+    Objects.requireNonNull(tablePath, "tablePath should not be null");
+    this.tablePath = tablePath;
+    return this;
+  }
+
+  public CarbonTableBuilder tableSchema(TableSchema tableSchema) {
+    Objects.requireNonNull(tableSchema, "tableSchema should not be null");
+    this.tableSchema = tableSchema;
+    return this;
+  }
+
+  public CarbonTable build() {
+    Objects.requireNonNull(tableName, "tableName should not be null");
+    Objects.requireNonNull(databaseName, "databaseName should not be null");
+    Objects.requireNonNull(tablePath, "tablePath should not be null");
+    Objects.requireNonNull(tableSchema, "tableSchema should not be null");
+
+    TableInfo tableInfo = new TableInfo();
+    tableInfo.setDatabaseName(databaseName);
+    tableInfo.setTableUniqueName(databaseName + "_" + tableName);
+    tableInfo.setFactTable(tableSchema);
+    tableInfo.setTablePath(tablePath);
+    tableInfo.setLastUpdatedTime(System.currentTimeMillis());
+    tableInfo.setDataMapSchemaList(new ArrayList<DataMapSchema>(0));
+    return CarbonTable.buildFromTableInfo(tableInfo);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
index 8fdfbab..fff1a74 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchema.java
@@ -294,4 +294,11 @@ public class TableSchema implements Serializable, Writable {
     return dataMapSchema;
   }
 
+  /**
+   * Create a {@link TableSchemaBuilder} to create {@link TableSchema}
+   */
+  public static TableSchemaBuilder builder() {
+    return new TableSchemaBuilder();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java
new file mode 100644
index 0000000..88774ec
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilder.java
@@ -0,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.apache.carbondata.core.metadata.schema.table;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
+import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Builder for {@link TableSchema}
+ */
+public class TableSchemaBuilder {
+
+  private int ordinal = 0;
+
+  private List<ColumnSchema> sortColumns = new LinkedList<>();
+
+  private List<ColumnSchema> otherColumns = new LinkedList<>();
+
+  public TableSchema build() {
+    TableSchema schema = new TableSchema();
+    schema.setTableId(UUID.randomUUID().toString());
+    schema.setPartitionInfo(null);
+    schema.setBucketingInfo(null);
+    SchemaEvolution schemaEvol = new SchemaEvolution();
+    schemaEvol.setSchemaEvolutionEntryList(new ArrayList<SchemaEvolutionEntry>());
+    schema.setSchemaEvalution(schemaEvol);
+    List<ColumnSchema> allColumns = new LinkedList<>(sortColumns);
+    allColumns.addAll(otherColumns);
+    schema.setListOfColumns(allColumns);
+    return schema;
+  }
+
+  public TableSchemaBuilder addColumn(StructField field, boolean isSortColumn) {
+    Objects.requireNonNull(field);
+    checkRepeatColumnName(field);
+    ColumnSchema newColumn = new ColumnSchema();
+    newColumn.setColumnName(field.getFieldName());
+    newColumn.setDataType(field.getDataType());
+    newColumn.setDimensionColumn(isSortColumn || field.getDataType() == DataTypes.STRING);
+    newColumn.setSchemaOrdinal(ordinal++);
+    newColumn.setColumnar(true);
+    newColumn.setColumnUniqueId(UUID.randomUUID().toString());
+    newColumn.setColumnReferenceId(newColumn.getColumnUniqueId());
+    newColumn.setEncodingList(createEncoding(field.getDataType(), isSortColumn));
+
+    if (isSortColumn) {
+      sortColumns.add(newColumn);
+    } else {
+      otherColumns.add(newColumn);
+    }
+    return this;
+  }
+
+  /**
+   * Throw exception if {@param field} name is repeated
+   */
+  private void checkRepeatColumnName(StructField field) {
+    for (ColumnSchema column : sortColumns) {
+      if (column.getColumnName().equalsIgnoreCase(field.getFieldName())) {
+        throw new IllegalArgumentException("column name already exists");
+      }
+    }
+    for (ColumnSchema column : otherColumns) {
+      if (column.getColumnName().equalsIgnoreCase(field.getFieldName())) {
+        throw new IllegalArgumentException("column name already exists");
+      }
+    }
+  }
+
+  private List<Encoding> createEncoding(DataType dataType, boolean isSortColumn) {
+    List<Encoding> encodings = new LinkedList<>();
+    if (dataType == DataTypes.TIMESTAMP || dataType == DataTypes.DATE) {
+      encodings.add(Encoding.DIRECT_DICTIONARY);
+    }
+    if (isSortColumn) {
+      encodings.add(Encoding.INVERTED_INDEX);
+    }
+    return encodings;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilderSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilderSuite.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilderSuite.java
new file mode 100644
index 0000000..83b65a0
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableBuilderSuite.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.metadata.schema.table;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class CarbonTableBuilderSuite {
+
+  TableSchema schema = CarbonTableTest.getTableSchema("t1");
+
+  @Test(expected = NullPointerException.class)
+  public void testNullTableName() {
+    TableSchema schema = CarbonTableTest.getTableSchema(null);
+    CarbonTable table = CarbonTable.builder()
+        .tableName(null)
+        .databaseName("db1")
+        .tableSchema(schema)
+        .tablePath("_temp")
+        .build();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullDbName() {
+    CarbonTable table = CarbonTable.builder()
+        .tableName(schema.getTableName())
+        .databaseName(null)
+        .tableSchema(schema)
+        .tablePath("_temp")
+        .build();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullSchema() {
+    CarbonTable table = CarbonTable.builder()
+        .tableName(schema.getTableName())
+        .databaseName("db1")
+        .tableSchema(null)
+        .tablePath("_temp")
+        .build();
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testNullTablePath() {
+    CarbonTable table = CarbonTable.builder()
+        .tableName(schema.getTableName())
+        .databaseName("db1")
+        .tableSchema(schema)
+        .tablePath(null)
+        .build();
+  }
+
+  @Test
+  public void testBuilder() {
+    CarbonTable table = CarbonTable.builder()
+        .tableName(schema.getTableName())
+        .databaseName("db1")
+        .tableSchema(schema)
+        .tablePath("_temp")
+        .build();
+    Assert.assertEquals(schema.getTableName(), table.getTableName());
+    Assert.assertEquals("db1", table.getDatabaseName());
+    Assert.assertEquals("_temp", table.getTablePath());
+    Assert.assertEquals(schema.getTableName(), table.getAbsoluteTableIdentifier().getTableName());
+    Assert.assertEquals("db1", table.getAbsoluteTableIdentifier().getDatabaseName());
+    Assert.assertEquals("_temp", table.getAbsoluteTableIdentifier().getTablePath());
+    Assert.assertEquals("db1_t1", table.getTableUniqueName());
+    Assert.assertEquals(schema, table.getTableInfo().getFactTable());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
index a47b7fd..0266df4 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
@@ -68,7 +68,7 @@ public class CarbonTableTest extends TestCase {
     assertTrue(carbonTable.getDimensionByName("carbonTestTable", "IMEI").equals(dimension));
   }
 
-  private ColumnSchema getColumnarDimensionColumn() {
+  static ColumnSchema getColumnarDimensionColumn() {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnar(true);
     dimColumn.setColumnName("IMEI");
@@ -83,7 +83,7 @@ public class CarbonTableTest extends TestCase {
     return dimColumn;
   }
 
-  private ColumnSchema getColumnarMeasureColumn() {
+  static ColumnSchema getColumnarMeasureColumn() {
     ColumnSchema dimColumn = new ColumnSchema();
     dimColumn.setColumnName("IMEI_COUNT");
     dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
@@ -91,23 +91,23 @@ public class CarbonTableTest extends TestCase {
     return dimColumn;
   }
 
-  private TableSchema getTableSchema() {
+  static TableSchema getTableSchema(String tableName) {
     TableSchema tableSchema = new TableSchema();
     List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
     columnSchemaList.add(getColumnarDimensionColumn());
     columnSchemaList.add(getColumnarMeasureColumn());
     tableSchema.setListOfColumns(columnSchemaList);
     tableSchema.setTableId(UUID.randomUUID().toString());
-    tableSchema.setTableName("carbonTestTable");
+    tableSchema.setTableName(tableName);
     return tableSchema;
   }
 
-  private TableInfo getTableInfo(long timeStamp) {
+  static private TableInfo getTableInfo(long timeStamp) {
     TableInfo info = new TableInfo();
     info.setDatabaseName("carbonTestDatabase");
     info.setLastUpdatedTime(timeStamp);
     info.setTableUniqueName("carbonTestDatabase_carbonTestTable");
-    info.setFactTable(getTableSchema());
+    info.setFactTable(getTableSchema("carbonTestTable"));
     info.setTablePath("testore");
     return info;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilderSuite.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilderSuite.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilderSuite.java
new file mode 100644
index 0000000..34fa920
--- /dev/null
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableSchemaBuilderSuite.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.metadata.schema.table;
+
+import java.util.List;
+
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TableSchemaBuilderSuite {
+
+  @Test(expected = NullPointerException.class)
+  public void testNullField() {
+    TableSchemaBuilder builder = TableSchema.builder();
+    builder.addColumn(null, true);
+  }
+
+  @Test
+  public void testBuilder() {
+    TableSchemaBuilder builder = TableSchema.builder();
+    builder.addColumn(new StructField("a", DataTypes.INT), true);
+    builder.addColumn(new StructField("b", DataTypes.DOUBLE), false);
+    TableSchema schema = builder.build();
+    Assert.assertEquals(2, schema.getListOfColumns().size());
+    List<ColumnSchema> columns = schema.getListOfColumns();
+    Assert.assertEquals("a", columns.get(0).getColumnName());
+    Assert.assertEquals("b", columns.get(1).getColumnName());
+  }
+
+  @Test(expected = IllegalArgumentException.class)
+  public void testRepeatedColumn() {
+    TableSchemaBuilder builder = TableSchema.builder();
+    builder.addColumn(new StructField("a", DataTypes.INT), true);
+    builder.addColumn(new StructField("a", DataTypes.DOUBLE), false);
+    TableSchema schema = builder.build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
index 49e4420..8d394db 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/DataLoadingUtil.scala
@@ -217,6 +217,51 @@ object DataLoadingUtil {
 
   /**
    * build CarbonLoadModel for data loading
+   * @param table CarbonTable object containing all metadata information for the table
+   *              like table name, table path, schema, etc
+   * @param options Load options from user input
+   * @return a new CarbonLoadModel instance
+   */
+  def buildCarbonLoadModelJava(
+      table: CarbonTable,
+      options: java.util.Map[String, String]
+  ): CarbonLoadModel = {
+    val carbonProperty: CarbonProperties = CarbonProperties.getInstance
+    val optionsFinal = getDataLoadingOptions(carbonProperty, options.asScala.toMap)
+    optionsFinal.put("sort_scope", "no_sort")
+    if (!options.containsKey("fileheader")) {
+      val csvHeader = table.getCreateOrderColumn(table.getTableName)
+        .asScala.map(_.getColName).mkString(",")
+      optionsFinal.put("fileheader", csvHeader)
+    }
+    val model = new CarbonLoadModel()
+    buildCarbonLoadModel(
+      table = table,
+      carbonProperty = carbonProperty,
+      options = options.asScala.toMap,
+      optionsFinal = optionsFinal,
+      carbonLoadModel = model,
+      hadoopConf = null)  // we have provided 'fileheader', so it can be null
+
+    // set default values
+    model.setTimestampformat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
+    model.setDateFormat(CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
+    model.setUseOnePass(options.asScala.getOrElse("onepass", "false").toBoolean)
+    model.setDictionaryServerHost(options.asScala.getOrElse("dicthost", null))
+    model.setDictionaryServerPort(options.asScala.getOrElse("dictport", "-1").toInt)
+    model
+  }
+
+  /**
+   * build CarbonLoadModel for data loading
+   * @param table CarbonTable object containing all metadata information for the table
+   *              like table name, table path, schema, etc
+   * @param carbonProperty Carbon property instance
+   * @param options Load options from user input
+   * @param optionsFinal Load options that populated with default values for optional options
+   * @param carbonLoadModel The output load model
+   * @param hadoopConf hadoopConf is needed to read CSV header if there 'fileheader' is not set in
+   *                   user provided load options
    */
   def buildCarbonLoadModel(
       table: CarbonTable,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0b9917d..520d770 100644
--- a/pom.xml
+++ b/pom.xml
@@ -99,6 +99,7 @@
     <module>core</module>
     <module>processing</module>
     <module>hadoop</module>
+    <module>store/sdk</module>
     <module>integration/spark-common</module>
     <module>integration/spark-common-test</module>
     <module>assembly</module>
@@ -571,6 +572,12 @@
       <id>include-all</id>
     </profile>
     <profile>
+      <id>store-sdk</id>
+      <modules>
+        <module>store/sdk</module>
+      </modules>
+    </profile>
+    <profile>
       <id>sdvtest</id>
       <modules>
         <module>integration/spark-common-cluster-test</module>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/store/sdk/pom.xml
----------------------------------------------------------------------
diff --git a/store/sdk/pom.xml b/store/sdk/pom.xml
new file mode 100644
index 0000000..6663683
--- /dev/null
+++ b/store/sdk/pom.xml
@@ -0,0 +1,130 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.carbondata</groupId>
+    <artifactId>carbondata-parent</artifactId>
+    <version>1.3.0-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+
+  <artifactId>carbondata-store-sdk</artifactId>
+  <name>Apache CarbonData :: Store SDK</name>
+
+  <properties>
+    <dev.path>${basedir}/../../dev</dev.path>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.carbondata</groupId>
+      <artifactId>carbondata-spark-common</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.scalatest</groupId>
+      <artifactId>scalatest_${scala.binary.version}</artifactId>
+      <version>2.2.1</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <resources>
+      <resource>
+        <directory>src/resources</directory>
+      </resource>
+      <resource>
+        <directory>.</directory>
+        <includes>
+          <include>CARBON_SPARK_INTERFACELogResource.properties</include>
+        </includes>
+      </resource>
+    </resources>
+    <plugins>
+      <plugin>
+        <groupId>org.scala-tools</groupId>
+        <artifactId>maven-scala-plugin</artifactId>
+        <version>2.15.2</version>
+        <executions>
+          <execution>
+            <id>compile</id>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+            <phase>compile</phase>
+          </execution>
+          <execution>
+            <id>testCompile</id>
+            <goals>
+              <goal>testCompile</goal>
+            </goals>
+            <phase>test</phase>
+          </execution>
+          <execution>
+            <phase>process-resources</phase>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <artifactId>maven-compiler-plugin</artifactId>
+        <configuration>
+          <source>1.7</source>
+          <target>1.7</target>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <version>2.18</version>
+        <!-- Note config is repeated in scalatest config -->
+        <configuration>
+          <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
+          <argLine>-Xmx3g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m</argLine>
+          <systemProperties>
+            <java.awt.headless>true</java.awt.headless>
+          </systemProperties>
+          <failIfNoTests>false</failIfNoTests>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.scalatest</groupId>
+        <artifactId>scalatest-maven-plugin</artifactId>
+        <version>1.0</version>
+        <!-- Note config is repeated in surefire config -->
+        <configuration>
+          <reportsDirectory>${project.build.directory}/surefire-reports</reportsDirectory>
+          <junitxml>.</junitxml>
+          <filereports>CarbonTestSuite.txt</filereports>
+          <argLine> ${argLine} -ea -Xmx3g -XX:MaxPermSize=512m -XX:ReservedCodeCacheSize=512m
+          </argLine>
+          <stderr />
+          <environmentVariables>
+          </environmentVariables>
+          <systemProperties>
+            <java.awt.headless>true</java.awt.headless>
+          </systemProperties>
+        </configuration>
+        <executions>
+          <execution>
+            <id>test</id>
+            <goals>
+              <goal>test</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+</project>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
new file mode 100644
index 0000000..daea733
--- /dev/null
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CSVCarbonWriter.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.sdk.file;
+
+import java.io.IOException;
+import java.util.Random;
+import java.util.UUID;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat;
+import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.TaskID;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+/**
+ * Implementation to write rows in CSV format to carbondata file.
+ */
+@InterfaceAudience.Developer
+@InterfaceStability.Unstable
+class CSVCarbonWriter extends CarbonWriter {
+
+  private RecordWriter<NullWritable, StringArrayWritable> recordWriter;
+  private TaskAttemptContext context;
+  private StringArrayWritable writable;
+
+  CSVCarbonWriter(CarbonLoadModel loadModel) throws IOException {
+    Configuration hadoopConf = new Configuration();
+    CarbonTableOutputFormat.setLoadModel(hadoopConf, loadModel);
+    CarbonTableOutputFormat format = new CarbonTableOutputFormat();
+    JobID jobId = new JobID(UUID.randomUUID().toString(), 0);
+    Random random = new Random();
+    TaskID task = new TaskID(jobId, TaskType.MAP, random.nextInt());
+    TaskAttemptID attemptID = new TaskAttemptID(task, random.nextInt());
+    TaskAttemptContextImpl context = new TaskAttemptContextImpl(hadoopConf, attemptID);
+    this.recordWriter = format.getRecordWriter(context);
+    this.context = context;
+    this.writable = new StringArrayWritable();
+  }
+
+  /**
+   * Write single row data, input row is of type String[]
+   */
+  @Override
+  public void write(Object object) throws IOException {
+    writable.set((String[]) object);
+    try {
+      recordWriter.write(NullWritable.get(), writable);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+
+  /**
+   * Flush and close the writer
+   */
+  @Override
+  public void close() throws IOException {
+    try {
+      recordWriter.close(context);
+    } catch (InterruptedException e) {
+      throw new IOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriter.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriter.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriter.java
new file mode 100644
index 0000000..e29aa18
--- /dev/null
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriter.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.sdk.file;
+
+import java.io.IOException;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * Writer to write row data to carbondata file. Call {@link #builder()} to get
+ * a build to create instance of writer.
+ */
+@InterfaceAudience.User
+@InterfaceStability.Unstable
+public abstract class CarbonWriter {
+
+  /**
+   * Write an object to the file, the format of the object depends on the
+   * implementation
+   */
+  public abstract void write(Object object) throws IOException;
+
+  /**
+   * Flush and close the writer
+   */
+  public abstract void close() throws IOException;
+
+  /**
+   * Create a {@link CarbonWriterBuilder} to build a {@link CarbonWriter}
+   */
+  public static CarbonWriterBuilder builder() {
+    return new CarbonWriterBuilder();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
new file mode 100644
index 0000000..51ca09c
--- /dev/null
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.sdk.file;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.carbondata.common.Strings;
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.converter.SchemaConverter;
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
+import org.apache.carbondata.core.metadata.datatype.StructField;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.TableSchema;
+import org.apache.carbondata.core.metadata.schema.table.TableSchemaBuilder;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.core.writer.ThriftWriter;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.spark.util.DataLoadingUtil;
+
+/**
+ * Biulder for {@link CarbonWriter}
+ */
+@InterfaceAudience.User
+@InterfaceStability.Unstable
+public class CarbonWriterBuilder {
+  private Schema schema;
+  private String path;
+  private String[] sortColumns;
+  private boolean persistSchemaFile;
+
+  public CarbonWriterBuilder withSchema(Schema schema) {
+    Objects.requireNonNull(schema, "schema should not be null");
+    this.schema = schema;
+    return this;
+  }
+
+  public CarbonWriterBuilder outputPath(String path) {
+    Objects.requireNonNull(path, "path should not be null");
+    this.path = path;
+    return this;
+  }
+
+  public CarbonWriterBuilder sortBy(String[] sortColumns) {
+    this.sortColumns = sortColumns;
+    return this;
+  }
+
+  public CarbonWriterBuilder partitionBy(String[] partitionColumns) {
+    throw new UnsupportedOperationException();
+  }
+
+  public CarbonWriterBuilder persistSchemaFile(boolean persist) {
+    this.persistSchemaFile = persist;
+    return this;
+  }
+
+  public CarbonWriterBuilder withBlockSize(int blockSize) {
+    if (blockSize <= 0) {
+      throw new IllegalArgumentException("blockSize should be greater than zero");
+    }
+    throw new UnsupportedOperationException();
+  }
+
+  public CarbonWriterBuilder withBlockletSize(int blockletSize) {
+    if (blockletSize <= 0) {
+      throw new IllegalArgumentException("blockletSize should be greater than zero");
+    }
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Build a {@link CSVCarbonWriter}, which accepts row in CSV format
+   */
+  public CarbonWriter buildWriterForCSVInput() throws IOException {
+    Objects.requireNonNull(schema, "schema should not be null");
+    Objects.requireNonNull(path, "path should not be null");
+
+    // build CarbonTable using schema
+    CarbonTable table = buildCarbonTable();
+    if (persistSchemaFile) {
+      // we are still using the traditional carbon table folder structure
+      persistSchemaFile(table, CarbonTablePath.getSchemaFilePath(path));
+    }
+
+    // build LoadModel
+    CarbonLoadModel loadModel = buildLoadModel(table);
+    return new CSVCarbonWriter(loadModel);
+  }
+
+  /**
+   * Build a {@link AvroCarbonWriter}, which accepts Avro object
+   * @return
+   * @throws IOException
+   */
+  public CarbonWriter buildWriterForAvroInput() throws IOException {
+    // TODO
+    throw new UnsupportedOperationException();
+  }
+
+  /**
+   * Build a {@link CarbonTable}
+   */
+  private CarbonTable buildCarbonTable() {
+    TableSchemaBuilder tableSchemaBuilder = TableSchema.builder();
+    List<String> sortColumnsList;
+    if (sortColumns != null) {
+      sortColumnsList = Arrays.asList(sortColumns);
+    } else {
+      sortColumnsList = new LinkedList<>();
+    }
+    for (Field field : schema.getFields()) {
+      tableSchemaBuilder.addColumn(
+          new StructField(field.getFieldName(), field.getDataType()),
+          sortColumnsList.contains(field.getFieldName()));
+    }
+    String tableName = "_tempTable";
+    String dbName = "_tempDB";
+    TableSchema schema = tableSchemaBuilder.build();
+    schema.setTableName(tableName);
+    CarbonTable table = CarbonTable.builder()
+        .tableName(schema.getTableName())
+        .databaseName(dbName)
+        .tablePath(path)
+        .tableSchema(schema)
+        .build();
+    return table;
+  }
+
+  /**
+   * Save the schema of the {@param table} to {@param persistFilePath}
+   * @param table table object containing schema
+   * @param persistFilePath absolute file path with file name
+   */
+  private void persistSchemaFile(CarbonTable table, String persistFilePath) throws IOException {
+    TableInfo tableInfo = table.getTableInfo();
+    String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(persistFilePath);
+    CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
+    SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
+    org.apache.carbondata.format.TableInfo thriftTableInfo =
+        schemaConverter.fromWrapperToExternalTableInfo(
+            tableInfo,
+            tableInfo.getDatabaseName(),
+            tableInfo.getFactTable().getTableName());
+    org.apache.carbondata.format.SchemaEvolutionEntry schemaEvolutionEntry =
+        new org.apache.carbondata.format.SchemaEvolutionEntry(
+            tableInfo.getLastUpdatedTime());
+    thriftTableInfo.getFact_table().getSchema_evolution().getSchema_evolution_history()
+        .add(schemaEvolutionEntry);
+    FileFactory.FileType fileType = FileFactory.getFileType(schemaMetadataPath);
+    if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
+      FileFactory.mkdirs(schemaMetadataPath, fileType);
+    }
+    ThriftWriter thriftWriter = new ThriftWriter(persistFilePath, false);
+    thriftWriter.open();
+    thriftWriter.write(thriftTableInfo);
+    thriftWriter.close();
+  }
+
+  /**
+   * Build a {@link CarbonLoadModel}
+   */
+  private CarbonLoadModel buildLoadModel(CarbonTable table) {
+    Map<String, String> options = new HashMap<>();
+    if (sortColumns != null) {
+      options.put("sort_columns", Strings.mkString(sortColumns, ","));
+    }
+    return DataLoadingUtil.buildCarbonLoadModelJava(table, options);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java
new file mode 100644
index 0000000..6742fa7
--- /dev/null
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.sdk.file;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+
+/**
+ * A field represent one column
+ */
+@InterfaceAudience.User
+@InterfaceStability.Unstable
+public class Field {
+
+  private String name;
+  private DataType type;
+
+  public Field(String name, String type) {
+    this.name = name;
+    if (type.equalsIgnoreCase("string")) {
+      this.type = DataTypes.STRING;
+    } else if (type.equalsIgnoreCase("date")) {
+      this.type = DataTypes.DATE;
+    } else if (type.equalsIgnoreCase("timestamp")) {
+      this.type = DataTypes.TIMESTAMP;
+    } else if (type.equalsIgnoreCase("boolean")) {
+      this.type = DataTypes.BOOLEAN;
+    } else if (type.equalsIgnoreCase("byte")) {
+      this.type = DataTypes.BYTE;
+    } else if (type.equalsIgnoreCase("short")) {
+      this.type = DataTypes.SHORT;
+    } else if (type.equalsIgnoreCase("int")) {
+      this.type = DataTypes.INT;
+    } else if (type.equalsIgnoreCase("long")) {
+      this.type = DataTypes.LONG;
+    } else if (type.equalsIgnoreCase("float")) {
+      this.type = DataTypes.FLOAT;
+    } else if (type.equalsIgnoreCase("double")) {
+      this.type = DataTypes.DOUBLE;
+    } else {
+      throw new IllegalArgumentException("unsupported data type: " + type);
+    }
+  }
+
+  public Field(String name, DataType type) {
+    this.name = name;
+    this.type = type;
+  }
+
+  public String getFieldName() {
+    return name;
+  }
+
+  public DataType getDataType() {
+    return type;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java
new file mode 100644
index 0000000..52a4611
--- /dev/null
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.sdk.file;
+
+import java.io.IOException;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+import com.google.gson.GsonBuilder;
+import com.google.gson.TypeAdapter;
+import com.google.gson.stream.JsonReader;
+import com.google.gson.stream.JsonWriter;
+
+/**
+ * A schema used to write and read data files
+ */
+@InterfaceAudience.User
+@InterfaceStability.Unstable
+public class Schema {
+
+  private Field[] fields;
+
+  public Schema(Field[] fields) {
+    this.fields = fields;
+  }
+
+  /**
+   * Create a Schema using JSON string, for example:
+   * [
+   *   {"name":"string"},
+   *   {"age":"int"}
+   * ]
+   */
+  public static Schema parseJson(String json) {
+    GsonBuilder gsonBuilder = new GsonBuilder();
+    gsonBuilder.registerTypeAdapter(Field.class, new TypeAdapter<Field>() {
+      @Override
+      public void write(JsonWriter out, Field field) throws IOException {
+        // noop
+      }
+
+      @Override
+      public Field read(JsonReader in) throws IOException {
+        in.beginObject();
+        Field field = new Field(in.nextName(), in.nextString());
+        in.endObject();
+        return field;
+      }
+    });
+
+    Field[] fields = gsonBuilder.create().fromJson(json, Field[].class);
+    return new Schema(fields);
+  }
+
+  public Field[] getFields() {
+    return fields;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/599b9ae9/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
new file mode 100644
index 0000000..531ec7c
--- /dev/null
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CSVCarbonWriterSuite.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.sdk.file;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.io.IOException;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test suite for {@link CSVCarbonWriter}
+ */
+public class CSVCarbonWriterSuite {
+
+  @Test
+  public void testWriteFiles() throws IOException {
+    String path = "./testWriteFiles";
+    FileUtils.deleteDirectory(new File(path));
+
+    Field[] fields = new Field[2];
+    fields[0] = new Field("name", DataTypes.STRING);
+    fields[1] = new Field("age", DataTypes.INT);
+
+    writeFilesAndVerify(new Schema(fields), path);
+
+    FileUtils.deleteDirectory(new File(path));
+  }
+
+  @Test
+  public void testWriteFilesJsonSchema() throws IOException {
+    String path = "./testWriteFilesJsonSchema";
+    FileUtils.deleteDirectory(new File(path));
+
+    String schema = new StringBuilder()
+        .append("[ \n")
+        .append("   {\"name\":\"string\"},\n")
+        .append("   {\"age\":\"int\"},\n")
+        .append("   {\"height\":\"double\"}\n")
+        .append("]")
+        .toString();
+
+    writeFilesAndVerify(Schema.parseJson(schema), path);
+
+    FileUtils.deleteDirectory(new File(path));
+  }
+
+  private void writeFilesAndVerify(Schema schema, String path) {
+    try {
+      CarbonWriter writer = CarbonWriter.builder()
+          .withSchema(schema)
+          .outputPath(path)
+          .buildWriterForCSVInput();
+
+      for (int i = 0; i < 100; i++) {
+        writer.write(new String[]{"robot" + i, String.valueOf(i), String.valueOf((double) i / 2)});
+      }
+      writer.close();
+    } catch (IOException e) {
+      e.printStackTrace();
+      Assert.fail(e.getMessage());
+    }
+
+    File segmentFolder = new File(CarbonTablePath.getSegmentPath(path, "null"));
+    Assert.assertTrue(segmentFolder.exists());
+
+    File[] dataFiles = segmentFolder.listFiles(new FileFilter() {
+      @Override public boolean accept(File pathname) {
+        return pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT);
+      }
+    });
+    Assert.assertNotNull(dataFiles);
+    Assert.assertEquals(1, dataFiles.length);
+  }
+
+  @Test
+  public void testAllPrimitiveDataType() {
+    // TODO: write all data type and read by CarbonRecordReader to verify the content
+  }
+
+  @Test
+  public void test2Blocklet() {
+    // TODO: write data with more than one blocklet
+  }
+
+  @Test
+  public void test2Block() {
+    // TODO: write data with more than one block
+  }
+
+  @Test
+  public void testSortColumns() {
+    // TODO: test sort column
+  }
+
+  @Test
+  public void testPartitionOutput() {
+    // TODO: test write data with partition
+  }
+
+  @Test
+  public void testSchemaPersistence() {
+    // TODO: verify schema file is persisted in specified location
+  }
+
+}


[28/49] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
index 11b3d43..527452a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -31,15 +31,14 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
@@ -63,21 +62,15 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    * entry count
    */
   private int entryCount;
-
   /**
    * return row
    */
-  private Object[] returnRow;
-  private int dimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
+  private IntermediateSortTempRow returnRow;
   private int readBufferSize;
   private String compressorName;
-  private Object[][] currentBuffer;
+  private IntermediateSortTempRow[] currentBuffer;
 
-  private Object[][] backupBuffer;
+  private IntermediateSortTempRow[] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -100,27 +93,21 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
   private int numberOfObjectRead;
 
-  private int nullSetWordsLength;
-
-  private Comparator<Object[]> comparator;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+  private Comparator<IntermediateSortTempRow> comparator;
   /**
    * Constructor to initialize
    */
   public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters parameters) {
     // set temp file
     this.tempFile = tempFile;
-    this.dimCnt = parameters.getDimColCount();
-    this.complexCnt = parameters.getComplexDimColCount();
-    this.measureCnt = parameters.getMeasureColCount();
-    this.isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
-    this.measureDataTypes = parameters.getMeasureDataType();
     this.readBufferSize = parameters.getBufferSize();
     this.compressorName = parameters.getSortTempCompressorName();
-
+    this.tableFieldStat = new TableFieldStat(parameters);
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
     this.executorService = Executors.newFixedThreadPool(1);
-    this.nullSetWordsLength = ((parameters.getMeasureColCount() - 1) >> 6) + 1;
-    comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
+    comparator = new IntermediateSortTempRowComparator(parameters.getNoDictionarySortColumn());
     initialize();
   }
 
@@ -169,11 +156,17 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    *
    * @throws CarbonSortKeyAndGroupByException problem while reading
    */
+  @Override
   public void readRow() throws CarbonSortKeyAndGroupByException {
     if (prefetch) {
       fillDataForPrefetch();
     } else {
-      this.returnRow = getRowFromStream();
+      try {
+        this.returnRow = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+        this.numberOfObjectRead++;
+      } catch (IOException e) {
+        throw new CarbonSortKeyAndGroupByException("Problems while reading row", e);
+      }
     }
   }
 
@@ -207,63 +200,22 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
   }
 
   /**
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
+   * get a batch of row, this interface is used in reading compressed sort temp files
+   *
+   * @param expected expected number in a batch
+   * @return a batch of row
+   * @throws IOException if error occurs while reading from stream
    */
-  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
-    Object[] row = new Object[dimCnt + measureCnt];
-    try {
-      int dimCount = 0;
-      for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
-        if (isNoDictionaryDimensionColumn[dimCount]) {
-          short aShort = stream.readShort();
-          byte[] col = new byte[aShort];
-          stream.readFully(col);
-          row[dimCount] = col;
-        } else {
-          int anInt = stream.readInt();
-          row[dimCount] = anInt;
-        }
-      }
-
-      // write complex dimensions here.
-      for (; dimCount < dimCnt; dimCount++) {
-        short aShort = stream.readShort();
-        byte[] col = new byte[aShort];
-        stream.readFully(col);
-        row[dimCount] = col;
-      }
-
-      long[] words = new long[nullSetWordsLength];
-      for (int i = 0; i < words.length; i++) {
-        words[i] = stream.readLong();
-      }
-
-      for (int mesCount = 0; mesCount < measureCnt; mesCount++) {
-        if (UnsafeCarbonRowPage.isSet(words, mesCount)) {
-          DataType dataType = measureDataTypes[mesCount];
-          if (dataType == DataTypes.SHORT) {
-            row[dimCount + mesCount] = stream.readShort();
-          } else if (dataType == DataTypes.INT) {
-            row[dimCount + mesCount] = stream.readInt();
-          } else if (dataType == DataTypes.LONG) {
-            row[dimCount + mesCount] = stream.readLong();
-          } else if (dataType == DataTypes.DOUBLE) {
-            row[dimCount + mesCount] = stream.readDouble();
-          } else if (DataTypes.isDecimal(dataType)) {
-            short aShort = stream.readShort();
-            byte[] bigDecimalInBytes = new byte[aShort];
-            stream.readFully(bigDecimalInBytes);
-            row[dimCount + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
-          } else {
-            throw new IllegalArgumentException("unsupported data type:" + dataType);
-          }
-        }
-      }
-      return row;
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException(e);
+  private IntermediateSortTempRow[] readBatchedRowFromStream(int expected)
+      throws IOException {
+    IntermediateSortTempRow[] holders = new IntermediateSortTempRow[expected];
+    for (int i = 0; i < expected; i++) {
+      IntermediateSortTempRow holder
+          = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+      holders[i] = holder;
     }
+    this.numberOfObjectRead += expected;
+    return holders;
   }
 
   /**
@@ -271,7 +223,7 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    *
    * @return row
    */
-  public Object[] getRow() {
+  public IntermediateSortTempRow getRow() {
     return this.returnRow;
   }
 
@@ -326,9 +278,7 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
 
   @Override public int hashCode() {
     int hash = 0;
-    hash += 31 * measureCnt;
-    hash += 31 * dimCnt;
-    hash += 31 * complexCnt;
+    hash += tableFieldStat.hashCode();
     hash += tempFile.hashCode();
     return hash;
   }
@@ -368,16 +318,12 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
   /**
    * This method will read the records from sort temp file and keep it in a buffer
    *
-   * @param numberOfRecords
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
+   * @param numberOfRecords number of records to be read
+   * @return batch of intermediate sort temp row
+   * @throws IOException if error occurs reading records from file
    */
-  private Object[][] prefetchRecordsFromFile(int numberOfRecords)
-      throws CarbonSortKeyAndGroupByException {
-    Object[][] records = new Object[numberOfRecords][];
-    for (int i = 0; i < numberOfRecords; i++) {
-      records[i] = getRowFromStream();
-    }
-    return records;
+  private IntermediateSortTempRow[] prefetchRecordsFromFile(int numberOfRecords)
+      throws IOException {
+    return readBatchedRowFromStream(numberOfRecords);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
index 4bbf61b..22673ff 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
@@ -21,25 +21,21 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
 import java.util.AbstractQueue;
-import java.util.Arrays;
 import java.util.PriorityQueue;
 import java.util.concurrent.Callable;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.SortTempChunkHolder;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeSortTempFileChunkHolder;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeIntermediateFileMerger implements Callable<Void> {
   /**
@@ -69,22 +65,13 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   private int totalNumberOfRecords;
 
   private SortParameters mergerParameters;
-
+  private TableFieldStat tableFieldStat;
   private File[] intermediateFiles;
-
   private File outPutFile;
 
-  private int dimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
   private int writeBufferSize;
   private String compressorName;
-
-  private long[] nullSetWords;
-
-  private ByteBuffer rowData;
+  private SortStepRowHandler sortStepRowHandler;
 
   private Throwable throwable;
 
@@ -97,16 +84,10 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
     this.fileCounter = intermediateFiles.length;
     this.intermediateFiles = intermediateFiles;
     this.outPutFile = outPutFile;
-    this.dimCnt = mergerParameters.getDimColCount();
-    this.complexCnt = mergerParameters.getComplexDimColCount();
-    this.measureCnt = mergerParameters.getMeasureColCount();
-    this.isNoDictionaryDimensionColumn = mergerParameters.getNoDictionaryDimnesionColumn();
-    this.measureDataTypes = mergerParameters.getMeasureDataType();
     this.writeBufferSize = mergerParameters.getBufferSize();
     this.compressorName = mergerParameters.getSortTempCompressorName();
-    this.nullSetWords = new long[((measureCnt - 1) >> 6) + 1];
-    // Take size of 2 MB for each row. I think it is high enough to use
-    rowData = ByteBuffer.allocate(2 * 1024 * 1024);
+    this.tableFieldStat = new TableFieldStat(mergerParameters);
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
   }
 
   @Override public Void call() throws Exception {
@@ -165,13 +146,14 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted record from file
+   * This method will be used to get sorted sort temp row from the sort temp files
    *
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] getSortedRecordFromFile() throws CarbonSortKeyAndGroupByException {
-    Object[] row = null;
+  private IntermediateSortTempRow getSortedRecordFromFile()
+      throws CarbonSortKeyAndGroupByException {
+    IntermediateSortTempRow row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will
@@ -235,7 +217,7 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
       this.recordHolderHeap.add(sortTempFileChunkHolder);
     }
 
-    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
+    LOGGER.info("Heap Size: " + this.recordHolderHeap.size());
   }
 
   /**
@@ -250,12 +232,12 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted row
+   * This method will be used to get the sorted sort temp row
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] next() throws CarbonSortKeyAndGroupByException {
+  private IntermediateSortTempRow next() throws CarbonSortKeyAndGroupByException {
     return getSortedRecordFromFile();
   }
 
@@ -272,82 +254,16 @@ public class UnsafeIntermediateFileMerger implements Callable<Void> {
   /**
    * Below method will be used to write data to file
    *
-   * @throws CarbonSortKeyAndGroupByException problem while writing
+   * @throws IOException problem while writing
    */
-  private void writeDataToFile(Object[] row) throws CarbonSortKeyAndGroupByException, IOException {
-    int dimCount = 0;
-    int size = 0;
-    for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
-      if (isNoDictionaryDimensionColumn[dimCount]) {
-        byte[] col = (byte[]) row[dimCount];
-        rowData.putShort((short) col.length);
-        size += 2;
-        rowData.put(col);
-        size += col.length;
-      } else {
-        rowData.putInt((int) row[dimCount]);
-        size += 4;
-      }
-    }
-
-    // write complex dimensions here.
-    int dimensionSize = dimCnt + complexCnt;
-    for (; dimCount < dimensionSize; dimCount++) {
-      byte[] col = (byte[]) row[dimCount];
-      rowData.putShort((short)col.length);
-      size += 2;
-      rowData.put(col);
-      size += col.length;
-    }
-    Arrays.fill(nullSetWords, 0);
-    int nullSetSize = nullSetWords.length * 8;
-    int nullLoc = size;
-    size += nullSetSize;
-    for (int mesCount = 0; mesCount < measureCnt; mesCount++) {
-      Object value = row[mesCount + dimensionSize];
-      if (null != value) {
-        DataType dataType = measureDataTypes[mesCount];
-        if (dataType == DataTypes.SHORT) {
-          rowData.putShort(size, (Short) value);
-          size += 2;
-        } else if (dataType == DataTypes.INT) {
-          rowData.putInt(size, (Integer) value);
-          size += 4;
-        } else if (dataType == DataTypes.LONG) {
-          rowData.putLong(size, (Long) value);
-          size += 8;
-        } else if (dataType == DataTypes.DOUBLE) {
-          rowData.putDouble(size, (Double) value);
-          size += 8;
-        } else if (DataTypes.isDecimal(dataType)) {
-          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(((BigDecimal) value));
-          rowData.putShort(size, (short) bigDecimalInBytes.length);
-          size += 2;
-          for (int i = 0; i < bigDecimalInBytes.length; i++) {
-            rowData.put(size++, bigDecimalInBytes[i]);
-          }
-        }
-        UnsafeCarbonRowPage.set(nullSetWords, mesCount);
-      } else {
-        UnsafeCarbonRowPage.unset(nullSetWords, mesCount);
-      }
-    }
-    for (int i = 0; i < nullSetWords.length; i++) {
-      rowData.putLong(nullLoc, nullSetWords[i]);
-      nullLoc += 8;
-    }
-    byte[] rowBytes = new byte[size];
-    rowData.position(0);
-    rowData.get(rowBytes);
-    stream.write(rowBytes);
-    rowData.clear();
+  private void writeDataToFile(IntermediateSortTempRow row) throws IOException {
+    sortStepRowHandler.writeIntermediateSortTempRowToOutputStream(row, stream);
   }
 
   private void finish() throws CarbonSortKeyAndGroupByException {
     clear();
     try {
       CarbonUtil.deleteFiles(intermediateFiles);
-      rowData.clear();
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while deleting the intermediate files");
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
index ce118d9..64f3c25 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
@@ -29,7 +29,8 @@ import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
-import org.apache.carbondata.processing.loading.sort.SortStepRowUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.SortTempChunkHolder;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeFinalMergePageHolder;
@@ -55,7 +56,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   private AbstractQueue<SortTempChunkHolder> recordHolderHeapLocal;
 
   private SortParameters parameters;
-  private SortStepRowUtil sortStepRowUtil;
+  private SortStepRowHandler sortStepRowHandler;
   /**
    * tempFileLocation
    */
@@ -68,7 +69,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   public UnsafeSingleThreadFinalSortFilesMerger(SortParameters parameters,
       String[] tempFileLocation) {
     this.parameters = parameters;
-    this.sortStepRowUtil = new SortStepRowUtil(parameters);
+    this.sortStepRowHandler = new SortStepRowHandler(parameters);
     this.tempFileLocation = tempFileLocation;
     this.tableName = parameters.getTableName();
   }
@@ -108,9 +109,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       LOGGER.info("Started adding first record from each page");
       for (final UnsafeCarbonRowPage rowPage : rowPages) {
 
-        SortTempChunkHolder sortTempFileChunkHolder = new UnsafeInmemoryHolder(rowPage,
-            parameters.getDimColCount() + parameters.getComplexDimColCount() + parameters
-                .getMeasureColCount(), parameters.getNumberOfSortColumns());
+        SortTempChunkHolder sortTempFileChunkHolder = new UnsafeInmemoryHolder(rowPage);
 
         // initialize
         sortTempFileChunkHolder.readRow();
@@ -121,9 +120,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       for (final UnsafeInMemoryIntermediateDataMerger merger : merges) {
 
         SortTempChunkHolder sortTempFileChunkHolder =
-            new UnsafeFinalMergePageHolder(merger, parameters.getNoDictionarySortColumn(),
-                parameters.getDimColCount() + parameters.getComplexDimColCount() + parameters
-                    .getMeasureColCount());
+            new UnsafeFinalMergePageHolder(merger, parameters.getNoDictionarySortColumn());
 
         // initialize
         sortTempFileChunkHolder.readRow();
@@ -142,7 +139,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
         recordHolderHeapLocal.add(sortTempFileChunkHolder);
       }
 
-      LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
+      LOGGER.info("Heap Size: " + this.recordHolderHeapLocal.size());
     } catch (Exception e) {
       LOGGER.error(e);
       throw new CarbonDataWriterException(e);
@@ -180,12 +177,14 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
   }
 
   /**
-   * This method will be used to get the sorted row
+   * This method will be used to get the sorted row in 3-parted format.
+   * The row will feed the following writer process step.
    *
    * @return sorted row
    */
   public Object[] next() {
-    return sortStepRowUtil.convertRow(getSortedRecordFromFile());
+    IntermediateSortTempRow sortTempRow =  getSortedRecordFromFile();
+    return sortStepRowHandler.convertIntermediateSortTempRowTo3Parted(sortTempRow);
   }
 
   /**
@@ -193,8 +192,8 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
    *
    * @return sorted record sorted record
    */
-  private Object[] getSortedRecordFromFile() throws CarbonDataWriterException {
-    Object[] row = null;
+  private IntermediateSortTempRow getSortedRecordFromFile() throws CarbonDataWriterException {
+    IntermediateSortTempRow row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index bfe38fd..b71612a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -389,7 +389,6 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
           noDictionarySortColumnMapping, 0, noDictionarySortColumnMapping.length);
     }
     sortParameters.setNoDictionarySortColumn(noDictionarySortColumnMapping);
-
     String[] sortTempFileLocation = CarbonDataProcessorUtil.arrayAppend(tempStoreLocation,
         CarbonCommonConstants.FILE_SEPARATOR, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
     finalMerger =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
index 04efa1f..c06819c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
@@ -21,7 +21,6 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.math.BigDecimal;
 import java.util.AbstractQueue;
 import java.util.PriorityQueue;
 import java.util.concurrent.Callable;
@@ -29,11 +28,9 @@ import java.util.concurrent.Callable;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 public class IntermediateFileMerger implements Callable<Void> {
@@ -68,17 +65,12 @@ public class IntermediateFileMerger implements Callable<Void> {
   private File[] intermediateFiles;
 
   private File outPutFile;
-  private int dimCnt;
-  private int noDictDimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private DataType[] measureDataTypes;
   private int writeBufferSize;
   private String compressorName;
 
   private Throwable throwable;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
   /**
    * IntermediateFileMerger Constructor
    */
@@ -88,14 +80,10 @@ public class IntermediateFileMerger implements Callable<Void> {
     this.fileCounter = intermediateFiles.length;
     this.intermediateFiles = intermediateFiles;
     this.outPutFile = outPutFile;
-    this.dimCnt = mergerParameters.getDimColCount();
-    this.noDictDimCnt = mergerParameters.getNoDictionaryCount();
-    this.complexCnt = mergerParameters.getComplexDimColCount();
-    this.measureCnt = mergerParameters.getMeasureColCount();
-    this.isNoDictionaryDimensionColumn = mergerParameters.getNoDictionaryDimnesionColumn();
-    this.measureDataTypes = mergerParameters.getMeasureDataType();
     this.writeBufferSize = mergerParameters.getBufferSize();
     this.compressorName = mergerParameters.getSortTempCompressorName();
+    this.tableFieldStat = new TableFieldStat(mergerParameters);
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
   }
 
   @Override public Void call() throws Exception {
@@ -154,13 +142,14 @@ public class IntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted record from file
+   * This method will be used to get the sorted sort temp row from sort temp file
    *
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] getSortedRecordFromFile() throws CarbonSortKeyAndGroupByException {
-    Object[] row = null;
+  private IntermediateSortTempRow getSortedRecordFromFile()
+      throws CarbonSortKeyAndGroupByException {
+    IntermediateSortTempRow row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will
@@ -227,7 +216,7 @@ public class IntermediateFileMerger implements Callable<Void> {
       this.recordHolderHeap.add(sortTempFileChunkHolder);
     }
 
-    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
+    LOGGER.info("Heap Size: " + this.recordHolderHeap.size());
   }
 
   /**
@@ -242,12 +231,12 @@ public class IntermediateFileMerger implements Callable<Void> {
   }
 
   /**
-   * This method will be used to get the sorted row
+   * This method will be used to get the sorted sort temp row
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] next() throws CarbonSortKeyAndGroupByException {
+  private IntermediateSortTempRow next() throws CarbonSortKeyAndGroupByException {
     return getSortedRecordFromFile();
   }
 
@@ -264,62 +253,10 @@ public class IntermediateFileMerger implements Callable<Void> {
   /**
    * Below method will be used to write data to file
    *
-   * @throws CarbonSortKeyAndGroupByException problem while writing
+   * @throws IOException problem while writing
    */
-  private void writeDataToFile(Object[] row) throws CarbonSortKeyAndGroupByException {
-    try {
-      int[] mdkArray = (int[]) row[0];
-      byte[][] nonDictArray = (byte[][]) row[1];
-      int mdkIndex = 0;
-      int nonDictKeyIndex = 0;
-      // write dictionary and non dictionary dimensions here.
-      for (boolean nodictinary : isNoDictionaryDimensionColumn) {
-        if (nodictinary) {
-          byte[] col = nonDictArray[nonDictKeyIndex++];
-          stream.writeShort(col.length);
-          stream.write(col);
-        } else {
-          stream.writeInt(mdkArray[mdkIndex++]);
-        }
-      }
-      // write complex
-      for (; nonDictKeyIndex < noDictDimCnt + complexCnt; nonDictKeyIndex++) {
-        byte[] col = nonDictArray[nonDictKeyIndex++];
-        stream.writeShort(col.length);
-        stream.write(col);
-      }
-      // write measure
-      int fieldIndex = 0;
-      for (int counter = 0; counter < measureCnt; counter++) {
-        if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
-          stream.write((byte) 1);
-          DataType dataType = measureDataTypes[counter];
-          if (dataType == DataTypes.BOOLEAN) {
-            stream.writeBoolean((boolean)NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.SHORT) {
-            stream.writeShort((short) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.INT) {
-            stream.writeInt((int) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.LONG) {
-            stream.writeLong((long) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (dataType == DataTypes.DOUBLE) {
-            stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
-          } else if (DataTypes.isDecimal(dataType)) {
-            byte[] bigDecimalInBytes = DataTypeUtil
-                .bigDecimalToByte((BigDecimal) NonDictionaryUtil.getMeasure(fieldIndex, row));
-            stream.writeInt(bigDecimalInBytes.length);
-            stream.write(bigDecimalInBytes);
-          } else {
-            throw new IllegalArgumentException("unsupported data type:" + dataType);
-          }
-        } else {
-          stream.write((byte) 0);
-        }
-        fieldIndex++;
-      }
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
-    }
+  private void writeDataToFile(IntermediateSortTempRow row) throws IOException {
+    sortStepRowHandler.writeIntermediateSortTempRowToOutputStream(row, stream);
   }
 
   private void finish() throws CarbonSortKeyAndGroupByException {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
new file mode 100644
index 0000000..9b6d1e8
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateSortTempRowComparator.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.sort.sortdata;
+
+import java.util.Comparator;
+
+import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+
+/**
+ * This class is used as comparator for comparing intermediate sort temp row
+ */
+public class IntermediateSortTempRowComparator implements Comparator<IntermediateSortTempRow> {
+  /**
+   * isSortColumnNoDictionary whether the sort column is not dictionary or not
+   */
+  private boolean[] isSortColumnNoDictionary;
+
+  /**
+   * @param isSortColumnNoDictionary isSortColumnNoDictionary
+   */
+  public IntermediateSortTempRowComparator(boolean[] isSortColumnNoDictionary) {
+    this.isSortColumnNoDictionary = isSortColumnNoDictionary;
+  }
+
+  /**
+   * Below method will be used to compare two sort temp row
+   */
+  public int compare(IntermediateSortTempRow rowA, IntermediateSortTempRow rowB) {
+    int diff = 0;
+    int dictIndex = 0;
+    int nonDictIndex = 0;
+
+    for (boolean isNoDictionary : isSortColumnNoDictionary) {
+
+      if (isNoDictionary) {
+        byte[] byteArr1 = rowA.getNoDictSortDims()[nonDictIndex];
+        byte[] byteArr2 = rowB.getNoDictSortDims()[nonDictIndex];
+        nonDictIndex++;
+
+        int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
+        if (difference != 0) {
+          return difference;
+        }
+      } else {
+        int dimFieldA = rowA.getDictSortDims()[dictIndex];
+        int dimFieldB = rowB.getDictSortDims()[dictIndex];
+        dictIndex++;
+
+        diff = dimFieldA - dimFieldB;
+        if (diff != 0) {
+          return diff;
+        }
+      }
+    }
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
index d2579d2..3f94533 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.java
@@ -40,14 +40,11 @@ public class NewRowComparator implements Comparator<Object[]> {
    */
   public int compare(Object[] rowA, Object[] rowB) {
     int diff = 0;
-
     int index = 0;
 
     for (boolean isNoDictionary : noDictionarySortColumnMaping) {
-
       if (isNoDictionary) {
         byte[] byteArr1 = (byte[]) rowA[index];
-
         byte[] byteArr2 = (byte[]) rowB[index];
 
         int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
@@ -57,6 +54,7 @@ public class NewRowComparator implements Comparator<Object[]> {
       } else {
         int dimFieldA = (int) rowA[index];
         int dimFieldB = (int) rowB[index];
+
         diff = dimFieldA - dimFieldB;
         if (diff != 0) {
           return diff;
@@ -65,7 +63,6 @@ public class NewRowComparator implements Comparator<Object[]> {
 
       index++;
     }
-
     return diff;
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
index e01b587..7538c92 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.java
@@ -29,7 +29,7 @@ public class NewRowComparatorForNormalDims implements Comparator<Object[]> {
   private int numberOfSortColumns;
 
   /**
-   * RowComparatorForNormalDims Constructor
+   * NewRowComparatorForNormalDims Constructor
    *
    * @param numberOfSortColumns
    */
@@ -46,7 +46,6 @@ public class NewRowComparatorForNormalDims implements Comparator<Object[]> {
     int diff = 0;
 
     for (int i = 0; i < numberOfSortColumns; i++) {
-
       int dimFieldA = (int)rowA[i];
       int dimFieldB = (int)rowB[i];
       diff = dimFieldA - dimFieldB;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
deleted file mode 100644
index 0ae0b93..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparator.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sort.sortdata;
-
-import java.nio.ByteBuffer;
-import java.util.Comparator;
-
-import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-
-public class RowComparator implements Comparator<Object[]> {
-  /**
-   * noDictionaryCount represent number of no dictionary cols
-   */
-  private int noDictionaryCount;
-
-  /**
-   * noDictionaryColMaping mapping of dictionary dimensions and no dictionary dimensions.
-   */
-  private boolean[] noDictionarySortColumnMaping;
-
-  /**
-   * @param noDictionarySortColumnMaping
-   * @param noDictionaryCount
-   */
-  public RowComparator(boolean[] noDictionarySortColumnMaping, int noDictionaryCount) {
-    this.noDictionaryCount = noDictionaryCount;
-    this.noDictionarySortColumnMaping = noDictionarySortColumnMaping;
-  }
-
-  /**
-   * Below method will be used to compare two mdkey
-   */
-  public int compare(Object[] rowA, Object[] rowB) {
-    int diff = 0;
-
-    int normalIndex = 0;
-    int noDictionaryindex = 0;
-
-    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
-
-      if (isNoDictionary) {
-        byte[] byteArr1 = (byte[]) rowA[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-
-        ByteBuffer buff1 = ByteBuffer.wrap(byteArr1);
-
-        // extract a high card dims from complete byte[].
-        NonDictionaryUtil
-            .extractSingleHighCardDims(byteArr1, noDictionaryindex, noDictionaryCount, buff1);
-
-        byte[] byteArr2 = (byte[]) rowB[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-
-        ByteBuffer buff2 = ByteBuffer.wrap(byteArr2);
-
-        // extract a high card dims from complete byte[].
-        NonDictionaryUtil
-            .extractSingleHighCardDims(byteArr2, noDictionaryindex, noDictionaryCount, buff2);
-
-        int difference = UnsafeComparer.INSTANCE.compareTo(buff1, buff2);
-        if (difference != 0) {
-          return difference;
-        }
-        noDictionaryindex++;
-      } else {
-        int dimFieldA = NonDictionaryUtil.getDimension(normalIndex, rowA);
-        int dimFieldB = NonDictionaryUtil.getDimension(normalIndex, rowB);
-        diff = dimFieldA - dimFieldB;
-        if (diff != 0) {
-          return diff;
-        }
-        normalIndex++;
-      }
-
-    }
-
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
deleted file mode 100644
index 0883ae1..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/RowComparatorForNormalDims.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.sort.sortdata;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-
-/**
- * This class is used as comparator for comparing dims which are non high cardinality dims.
- * Here the dims will be in form of int[] (surrogates) so directly comparing the integers.
- */
-public class RowComparatorForNormalDims implements Comparator<Object[]> {
-  /**
-   * dimension count
-   */
-  private int numberOfSortColumns;
-
-  /**
-   * RowComparatorForNormalDims Constructor
-   *
-   * @param numberOfSortColumns
-   */
-  public RowComparatorForNormalDims(int numberOfSortColumns) {
-    this.numberOfSortColumns = numberOfSortColumns;
-  }
-
-  /**
-   * Below method will be used to compare two surrogate keys
-   *
-   * @see Comparator#compare(Object, Object)
-   */
-  public int compare(Object[] rowA, Object[] rowB) {
-    int diff = 0;
-
-    for (int i = 0; i < numberOfSortColumns; i++) {
-
-      int dimFieldA = NonDictionaryUtil.getDimension(i, rowA);
-      int dimFieldB = NonDictionaryUtil.getDimension(i, rowB);
-
-      diff = dimFieldA - dimFieldB;
-      if (diff != 0) {
-        return diff;
-      }
-    }
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
index 88695b9..a4ac0ea 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
@@ -37,6 +37,8 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
@@ -71,12 +73,12 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    * tableName
    */
   private String tableName;
-
+  private SortParameters sortParameters;
+  private SortStepRowHandler sortStepRowHandler;
   /**
    * tempFileLocation
    */
   private String[] tempFileLocation;
-  private SortParameters sortParameters;
 
   private int maxThreadForSorting;
 
@@ -89,6 +91,7 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
     this.tempFileLocation = tempFileLocation;
     this.tableName = tableName;
     this.sortParameters = sortParameters;
+    this.sortStepRowHandler = new SortStepRowHandler(sortParameters);
     try {
       maxThreadForSorting = Integer.parseInt(CarbonProperties.getInstance()
           .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD,
@@ -107,8 +110,7 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    */
   public void startFinalMerge() throws CarbonDataWriterException {
     List<File> filesToMerge = getFilesToMergeSort();
-    if (filesToMerge.size() == 0)
-    {
+    if (filesToMerge.size() == 0) {
       LOGGER.info("No file to merge in final merge stage");
       return;
     }
@@ -125,11 +127,9 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
 
     // get all the merged files
     List<File> files = new ArrayList<File>(tempFileLocation.length);
-    for (String tempLoc : tempFileLocation)
-    {
+    for (String tempLoc : tempFileLocation) {
       File[] subFiles = new File(tempLoc).listFiles(fileFilter);
-      if (null != subFiles && subFiles.length > 0)
-      {
+      if (null != subFiles && subFiles.length > 0) {
         files.addAll(Arrays.asList(subFiles));
       }
     }
@@ -226,13 +226,14 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
   }
 
   /**
-   * This method will be used to get the sorted row
+   * This method will be used to get the sorted sort temp row from the sort temp files
    *
    * @return sorted row
    * @throws CarbonSortKeyAndGroupByException
    */
   public Object[] next() {
-    return getSortedRecordFromFile();
+    IntermediateSortTempRow sortTempRow = getSortedRecordFromFile();
+    return sortStepRowHandler.convertIntermediateSortTempRowTo3Parted(sortTempRow);
   }
 
   /**
@@ -241,8 +242,8 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
    * @return sorted record sorted record
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] getSortedRecordFromFile() throws CarbonDataWriterException {
-    Object[] row = null;
+  private IntermediateSortTempRow getSortedRecordFromFile() throws CarbonDataWriterException {
+    IntermediateSortTempRow row = null;
 
     // poll the top object from heap
     // heap maintains binary tree which is based on heap condition that will

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
index 57a19bd..c7efbd9 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.processing.sort.sortdata;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
-import java.math.BigDecimal;
+import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
@@ -32,12 +32,10 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
@@ -69,7 +67,8 @@ public class SortDataRows {
   private Semaphore semaphore;
 
   private SortParameters parameters;
-
+  private SortStepRowHandler sortStepRowHandler;
+  private ThreadLocal<ByteBuffer> rowBuffer;
   private int sortBufferSize;
 
   private SortIntermediateFileMerger intermediateFileMerger;
@@ -79,7 +78,7 @@ public class SortDataRows {
   public SortDataRows(SortParameters parameters,
       SortIntermediateFileMerger intermediateFileMerger) {
     this.parameters = parameters;
-
+    this.sortStepRowHandler = new SortStepRowHandler(parameters);
     this.intermediateFileMerger = intermediateFileMerger;
 
     int batchSize = CarbonProperties.getInstance().getBatchSize();
@@ -87,6 +86,12 @@ public class SortDataRows {
     this.sortBufferSize = Math.max(parameters.getSortBufferSize(), batchSize);
     // observer of writing file in thread
     this.threadStatusObserver = new ThreadStatusObserver();
+    this.rowBuffer = new ThreadLocal<ByteBuffer>() {
+      @Override protected ByteBuffer initialValue() {
+        byte[] backedArray = new byte[2 * 1024 * 1024];
+        return ByteBuffer.wrap(backedArray);
+      }
+    };
   }
 
   /**
@@ -130,8 +135,7 @@ public class SortDataRows {
         semaphore.acquire();
         dataSorterAndWriterExecutorService.execute(new DataSorterAndWriter(recordHolderListLocal));
       } catch (InterruptedException e) {
-        LOGGER.error(e,
-            "exception occurred while trying to acquire a semaphore lock: ");
+        LOGGER.error(e, "exception occurred while trying to acquire a semaphore lock: ");
         throw new CarbonSortKeyAndGroupByException(e);
       }
       // create the new holder Array
@@ -158,7 +162,7 @@ public class SortDataRows {
         }
         intermediateFileMerger.startMergingIfPossible();
         Object[][] recordHolderListLocal = recordHolderList;
-        sizeLeft = sortBufferSize - entryCount ;
+        sizeLeft = sortBufferSize - entryCount;
         if (sizeLeft > 0) {
           System.arraycopy(rowBatch, 0, recordHolderListLocal, entryCount, sizeLeft);
         }
@@ -212,7 +216,6 @@ public class SortDataRows {
           locationChosen + File.separator + parameters.getTableName() +
               System.nanoTime() + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
       writeDataToFile(recordHolderList, this.entryCount, file);
-
     }
 
     startFileBasedMerge();
@@ -220,7 +223,7 @@ public class SortDataRows {
   }
 
   /**
-   * Below method will be used to write data to file
+   * Below method will be used to write data to sort temp file
    *
    * @throws CarbonSortKeyAndGroupByException problem while writing
    */
@@ -233,60 +236,9 @@ public class SortDataRows {
           parameters.getFileWriteBufferSize(), parameters.getSortTempCompressorName());
       // write number of entries to the file
       stream.writeInt(entryCountLocal);
-      int complexDimColCount = parameters.getComplexDimColCount();
-      int dimColCount = parameters.getDimColCount() + complexDimColCount;
-      DataType[] type = parameters.getMeasureDataType();
-      boolean[] noDictionaryDimnesionMapping = parameters.getNoDictionaryDimnesionColumn();
-      Object[] row = null;
       for (int i = 0; i < entryCountLocal; i++) {
-        // get row from record holder list
-        row = recordHolderList[i];
-        int dimCount = 0;
-        // write dictionary and non dictionary dimensions here.
-        for (; dimCount < noDictionaryDimnesionMapping.length; dimCount++) {
-          if (noDictionaryDimnesionMapping[dimCount]) {
-            byte[] col = (byte[]) row[dimCount];
-            stream.writeShort(col.length);
-            stream.write(col);
-          } else {
-            stream.writeInt((int)row[dimCount]);
-          }
-        }
-        // write complex dimensions here.
-        for (; dimCount < dimColCount; dimCount++) {
-          byte[] value = (byte[])row[dimCount];
-          stream.writeShort(value.length);
-          stream.write(value);
-        }
-        // as measures are stored in separate array.
-        for (int mesCount = 0;
-             mesCount < parameters.getMeasureColCount(); mesCount++) {
-          Object value = row[mesCount + dimColCount];
-          if (null != value) {
-            stream.write((byte) 1);
-            DataType dataType = type[mesCount];
-            if (dataType == DataTypes.BOOLEAN) {
-              stream.writeBoolean((boolean) value);
-            } else if (dataType == DataTypes.SHORT) {
-              stream.writeShort((Short) value);
-            } else if (dataType == DataTypes.INT) {
-              stream.writeInt((Integer) value);
-            } else if (dataType == DataTypes.LONG) {
-              stream.writeLong((Long) value);
-            } else if (dataType == DataTypes.DOUBLE) {
-              stream.writeDouble((Double) value);
-            } else if (DataTypes.isDecimal(dataType)) {
-              BigDecimal val = (BigDecimal) value;
-              byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
-              stream.writeInt(bigDecimalInBytes.length);
-              stream.write(bigDecimalInBytes);
-            } else {
-              throw new IllegalArgumentException("unsupported data type:" + type[mesCount]);
-            }
-          } else {
-            stream.write((byte) 0);
-          }
-        }
+        sortStepRowHandler.writeRawRowAsIntermediateSortTempRowToOutputStream(
+            recordHolderList[i], stream, rowBuffer.get());
       }
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
@@ -301,7 +253,7 @@ public class SortDataRows {
    *
    * @throws CarbonSortKeyAndGroupByException
    */
-  public void deleteSortLocationIfExists() throws CarbonSortKeyAndGroupByException {
+  private void deleteSortLocationIfExists() throws CarbonSortKeyAndGroupByException {
     CarbonDataProcessorUtil.deleteSortLocationIfExists(parameters.getTempFileLocation());
   }
 
@@ -380,7 +332,8 @@ public class SortDataRows {
         // intermediate merging of sort temp files will be triggered
         intermediateFileMerger.addFileToMerge(sortTempFile);
         LOGGER.info("Time taken to sort and write sort temp file " + sortTempFile + " is: " + (
-            System.currentTimeMillis() - startTime));
+            System.currentTimeMillis() - startTime) + ", sort temp file size in MB is "
+            + sortTempFile.length() * 0.1 * 10 / 1024 / 1024);
       } catch (Throwable e) {
         try {
           threadStatusObserver.notifyFailed(e);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
index d726539..7e221a7 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
@@ -21,6 +21,7 @@ import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.Comparator;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -30,14 +31,11 @@ import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHolder> {
@@ -71,20 +69,13 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   /**
    * return row
    */
-  private Object[] returnRow;
-  private int dimCnt;
-  private int noDictDimCnt;
-  private int complexCnt;
-  private int measureCnt;
-  private boolean[] isNoDictionaryDimensionColumn;
-  private boolean[] isNoDictionarySortColumn;
-  private DataType[] measureDataTypes;
+  private IntermediateSortTempRow returnRow;
   private int readBufferSize;
   private String compressorName;
 
-  private Object[][] currentBuffer;
+  private IntermediateSortTempRow[] currentBuffer;
 
-  private Object[][] backupBuffer;
+  private IntermediateSortTempRow[] backupBuffer;
 
   private boolean isBackupFilled;
 
@@ -104,7 +95,9 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
    * totalRecordFetch
    */
   private int totalRecordFetch;
-
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+  private Comparator<IntermediateSortTempRow> comparator;
   /**
    * Constructor to initialize
    *
@@ -115,16 +108,12 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   public SortTempFileChunkHolder(File tempFile, SortParameters sortParameters, String tableName) {
     // set temp file
     this.tempFile = tempFile;
-    this.dimCnt = sortParameters.getDimColCount();
-    this.noDictDimCnt = sortParameters.getNoDictionaryCount();
-    this.complexCnt = sortParameters.getComplexDimColCount();
-    this.measureCnt = sortParameters.getMeasureColCount();
-    this.isNoDictionaryDimensionColumn = sortParameters.getNoDictionaryDimnesionColumn();
-    this.isNoDictionarySortColumn = sortParameters.getNoDictionarySortColumn();
-    this.measureDataTypes = sortParameters.getMeasureDataType();
     this.readBufferSize = sortParameters.getBufferSize();
     this.compressorName = sortParameters.getSortTempCompressorName();
-
+    this.tableFieldStat = new TableFieldStat(sortParameters);
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
+    this.comparator = new IntermediateSortTempRowComparator(
+        tableFieldStat.getIsSortColNoDictFlags());
     this.executorService = Executors
         .newFixedThreadPool(1, new CarbonThreadFactory("SafeSortTempChunkHolderPool:" + tableName));
   }
@@ -178,7 +167,12 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
     if (prefetch) {
       fillDataForPrefetch();
     } else {
-      this.returnRow = getRowFromStream();
+      try {
+        this.returnRow = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+        this.numberOfObjectRead++;
+      } catch (IOException e) {
+        throw new CarbonSortKeyAndGroupByException("Problem while reading rows", e);
+      }
     }
   }
 
@@ -212,86 +206,28 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   }
 
   /**
-   * Reads row from file
+   * Read a batch of row from stream
+   *
    * @return Object[]
-   * @throws CarbonSortKeyAndGroupByException
+   * @throws IOException if error occurs while reading from stream
    */
-  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
-    // create new row of size 3 (1 for dims , 1 for high card , 1 for measures)
-
-    Object[] holder = new Object[3];
-    int index = 0;
-    int nonDicIndex = 0;
-    int[] dim = new int[dimCnt - noDictDimCnt];
-    byte[][] nonDicArray = new byte[noDictDimCnt + complexCnt][];
-    Object[] measures = new Object[measureCnt];
-    try {
-      // read dimension values
-      for (int i = 0; i < isNoDictionaryDimensionColumn.length; i++) {
-        if (isNoDictionaryDimensionColumn[i]) {
-          short len = stream.readShort();
-          byte[] array = new byte[len];
-          stream.readFully(array);
-          nonDicArray[nonDicIndex++] = array;
-        } else {
-          dim[index++] = stream.readInt();
-        }
-      }
-
-      for (int i = 0; i < complexCnt; i++) {
-        short len = stream.readShort();
-        byte[] array = new byte[len];
-        stream.readFully(array);
-        nonDicArray[nonDicIndex++] = array;
-      }
-
-      index = 0;
-      // read measure values
-      for (int i = 0; i < measureCnt; i++) {
-        if (stream.readByte() == 1) {
-          DataType dataType = measureDataTypes[i];
-          if (dataType == DataTypes.BOOLEAN) {
-            measures[index++] = stream.readBoolean();
-          } else if (dataType == DataTypes.SHORT) {
-            measures[index++] = stream.readShort();
-          } else if (dataType == DataTypes.INT) {
-            measures[index++] = stream.readInt();
-          } else if (dataType == DataTypes.LONG) {
-            measures[index++] = stream.readLong();
-          } else if (dataType == DataTypes.DOUBLE) {
-            measures[index++] = stream.readDouble();
-          } else if (DataTypes.isDecimal(dataType)) {
-            int len = stream.readInt();
-            byte[] buff = new byte[len];
-            stream.readFully(buff);
-            measures[index++] = DataTypeUtil.byteToBigDecimal(buff);
-          } else {
-            throw new IllegalArgumentException("unsupported data type:" + dataType);
-          }
-        } else {
-          measures[index++] = null;
-        }
-      }
-
-      NonDictionaryUtil.prepareOutObj(holder, dim, nonDicArray, measures);
-
-      // increment number if record read
-      this.numberOfObjectRead++;
-    } catch (IOException e) {
-      LOGGER.error("Problme while reading the madkey fom sort temp file");
-      throw new CarbonSortKeyAndGroupByException("Problem while reading the sort temp file ", e);
+  private IntermediateSortTempRow[] readBatchedRowFromStream(int expected) throws IOException {
+    IntermediateSortTempRow[] holders = new IntermediateSortTempRow[expected];
+    for (int i = 0; i < expected; i++) {
+      IntermediateSortTempRow holder
+          = sortStepRowHandler.readIntermediateSortTempRowFromInputStream(stream);
+      holders[i] = holder;
     }
-
-    //return out row
-    return holder;
+    this.numberOfObjectRead += expected;
+    return holders;
   }
 
   /**
-   * below method will be used to get the row
+   * below method will be used to get the sort temp row
    *
    * @return row
    */
-  public Object[] getRow() {
+  public IntermediateSortTempRow getRow() {
     return this.returnRow;
   }
 
@@ -330,31 +266,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   }
 
   @Override public int compareTo(SortTempFileChunkHolder other) {
-    int diff = 0;
-    int index = 0;
-    int noDictionaryIndex = 0;
-    int[] leftMdkArray = (int[]) returnRow[0];
-    int[] rightMdkArray = (int[]) other.returnRow[0];
-    byte[][] leftNonDictArray = (byte[][]) returnRow[1];
-    byte[][] rightNonDictArray = (byte[][]) other.returnRow[1];
-    for (boolean isNoDictionary : isNoDictionarySortColumn) {
-      if (isNoDictionary) {
-        diff = UnsafeComparer.INSTANCE
-            .compareTo(leftNonDictArray[noDictionaryIndex], rightNonDictArray[noDictionaryIndex]);
-        if (diff != 0) {
-          return diff;
-        }
-        noDictionaryIndex++;
-      } else {
-        diff = leftMdkArray[index] - rightMdkArray[index];
-        if (diff != 0) {
-          return diff;
-        }
-        index++;
-      }
-
-    }
-    return diff;
+    return comparator.compare(returnRow, other.getRow());
   }
 
   @Override public boolean equals(Object obj) {
@@ -372,9 +284,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
 
   @Override public int hashCode() {
     int hash = 0;
-    hash += 31 * measureCnt;
-    hash += 31 * dimCnt;
-    hash += 31 * complexCnt;
+    hash += tableFieldStat.hashCode();
     hash += tempFile.hashCode();
     return hash;
   }
@@ -414,16 +324,12 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   /**
    * This method will read the records from sort temp file and keep it in a buffer
    *
-   * @param numberOfRecords
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
+   * @param numberOfRecords number of records to be read
+   * @return batch of intermediate sort temp row
+   * @throws IOException if error occurs while reading reading records
    */
-  private Object[][] prefetchRecordsFromFile(int numberOfRecords)
-      throws CarbonSortKeyAndGroupByException {
-    Object[][] records = new Object[numberOfRecords][];
-    for (int i = 0; i < numberOfRecords; i++) {
-      records[i] = getRowFromStream();
-    }
-    return records;
+  private IntermediateSortTempRow[] prefetchRecordsFromFile(int numberOfRecords)
+      throws IOException {
+    return readBatchedRowFromStream(numberOfRecords);
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
new file mode 100644
index 0000000..0d1303a
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
@@ -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.apache.carbondata.processing.sort.sortdata;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+
+/**
+ * This class is used to hold field information for a table during data loading. These information
+ * will be used to convert/construct/destruct row in sort process step. Because complex field is
+ * processed the same as no-dict-no-sort-simple-dimension, so we treat them as the same and use
+ * `no-dict-no-sort-dim` related variable to represent them in this class.
+ */
+public class TableFieldStat implements Serializable {
+  private static final long serialVersionUID = 201712070950L;
+  private int dictSortDimCnt = 0;
+  private int dictNoSortDimCnt = 0;
+  private int noDictSortDimCnt = 0;
+  private int noDictNoSortDimCnt = 0;
+  // whether sort column is of dictionary type or not
+  private boolean[] isSortColNoDictFlags;
+  private int measureCnt;
+  private DataType[] measureDataType;
+
+  // indices for dict & sort dimension columns
+  private int[] dictSortDimIdx;
+  // indices for dict & no-sort dimension columns
+  private int[] dictNoSortDimIdx;
+  // indices for no-dict & sort dimension columns
+  private int[] noDictSortDimIdx;
+  // indices for no-dict & no-sort dimension columns, including complex columns
+  private int[] noDictNoSortDimIdx;
+  // indices for measure columns
+  private int[] measureIdx;
+
+  public TableFieldStat(SortParameters sortParameters) {
+    int noDictDimCnt = sortParameters.getNoDictionaryCount();
+    int complexDimCnt = sortParameters.getComplexDimColCount();
+    int dictDimCnt = sortParameters.getDimColCount() - noDictDimCnt;
+    this.isSortColNoDictFlags = sortParameters.getNoDictionarySortColumn();
+    int sortColCnt = isSortColNoDictFlags.length;
+    for (boolean flag : isSortColNoDictFlags) {
+      if (flag) {
+        noDictSortDimCnt++;
+      } else {
+        dictSortDimCnt++;
+      }
+    }
+    this.measureCnt = sortParameters.getMeasureColCount();
+    this.measureDataType = sortParameters.getMeasureDataType();
+
+    // be careful that the default value is 0
+    this.dictSortDimIdx = new int[dictSortDimCnt];
+    this.dictNoSortDimIdx = new int[dictDimCnt - dictSortDimCnt];
+    this.noDictSortDimIdx = new int[noDictSortDimCnt];
+    this.noDictNoSortDimIdx = new int[noDictDimCnt + complexDimCnt - noDictSortDimCnt];
+    this.measureIdx = new int[measureCnt];
+
+    int tmpNoDictSortCnt = 0;
+    int tmpNoDictNoSortCnt = 0;
+    int tmpDictSortCnt = 0;
+    int tmpDictNoSortCnt = 0;
+    boolean[] isDimNoDictFlags = sortParameters.getNoDictionaryDimnesionColumn();
+
+    for (int i = 0; i < isDimNoDictFlags.length; i++) {
+      if (isDimNoDictFlags[i]) {
+        if (i < sortColCnt && isSortColNoDictFlags[i]) {
+          noDictSortDimIdx[tmpNoDictSortCnt++] = i;
+        } else {
+          noDictNoSortDimIdx[tmpNoDictNoSortCnt++] = i;
+        }
+      } else {
+        if (i < sortColCnt && !isSortColNoDictFlags[i]) {
+          dictSortDimIdx[tmpDictSortCnt++] = i;
+        } else {
+          dictNoSortDimIdx[tmpDictNoSortCnt++] = i;
+        }
+      }
+    }
+    dictNoSortDimCnt = tmpDictNoSortCnt;
+
+    int base = isDimNoDictFlags.length;
+    // adding complex dimension columns
+    for (int i = 0; i < complexDimCnt; i++) {
+      noDictNoSortDimIdx[tmpNoDictNoSortCnt++] = base + i;
+    }
+    noDictNoSortDimCnt = tmpNoDictNoSortCnt;
+
+    base += complexDimCnt;
+    // indices for measure columns
+    for (int i = 0; i < measureCnt; i++) {
+      measureIdx[i] = base + i;
+    }
+  }
+
+  public int getDictSortDimCnt() {
+    return dictSortDimCnt;
+  }
+
+  public int getDictNoSortDimCnt() {
+    return dictNoSortDimCnt;
+  }
+
+  public int getNoDictSortDimCnt() {
+    return noDictSortDimCnt;
+  }
+
+  public int getNoDictNoSortDimCnt() {
+    return noDictNoSortDimCnt;
+  }
+
+  public boolean[] getIsSortColNoDictFlags() {
+    return isSortColNoDictFlags;
+  }
+
+  public int getMeasureCnt() {
+    return measureCnt;
+  }
+
+  public DataType[] getMeasureDataType() {
+    return measureDataType;
+  }
+
+  public int[] getDictSortDimIdx() {
+    return dictSortDimIdx;
+  }
+
+  public int[] getDictNoSortDimIdx() {
+    return dictNoSortDimIdx;
+  }
+
+  public int[] getNoDictSortDimIdx() {
+    return noDictSortDimIdx;
+  }
+
+  public int[] getNoDictNoSortDimIdx() {
+    return noDictNoSortDimIdx;
+  }
+
+  public int[] getMeasureIdx() {
+    return measureIdx;
+  }
+
+  @Override public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof TableFieldStat)) return false;
+    TableFieldStat that = (TableFieldStat) o;
+    return dictSortDimCnt == that.dictSortDimCnt
+        && dictNoSortDimCnt == that.dictNoSortDimCnt
+        && noDictSortDimCnt == that.noDictSortDimCnt
+        && noDictNoSortDimCnt == that.noDictNoSortDimCnt
+        && measureCnt == that.measureCnt;
+  }
+
+  @Override public int hashCode() {
+    return Objects.hash(dictSortDimCnt, dictNoSortDimCnt, noDictSortDimCnt,
+        noDictNoSortDimCnt, measureCnt);
+  }
+}
\ No newline at end of file


[29/49] carbondata git commit: [CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

Posted by ja...@apache.org.
[CARBONDATA-2018][DataLoad] Optimization in reading/writing for sort temp row

Pick up the no-sort fields in the row and pack them as bytes array and skip parsing them during merge sort to reduce CPU consumption

This closes #1792


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

Branch: refs/heads/carbonstore-rebase4
Commit: 2d77936814c0f6bfa35b0506dd558729e3cacfea
Parents: 5cc6d36
Author: xuchuanyin <xu...@hust.edu.cn>
Authored: Thu Feb 8 14:35:14 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Feb 27 00:30:22 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/util/NonDictionaryUtil.java |  67 +--
 .../presto/util/CarbonDataStoreCreator.scala    |   1 -
 .../load/DataLoadProcessorStepOnSpark.scala     |   6 +-
 .../loading/row/IntermediateSortTempRow.java    | 117 +++++
 .../loading/sort/SortStepRowHandler.java        | 466 +++++++++++++++++++
 .../loading/sort/SortStepRowUtil.java           | 103 ----
 .../sort/unsafe/UnsafeCarbonRowPage.java        | 331 ++-----------
 .../loading/sort/unsafe/UnsafeSortDataRows.java |  57 +--
 .../unsafe/comparator/UnsafeRowComparator.java  |  95 ++--
 .../UnsafeRowComparatorForNormalDIms.java       |  59 ---
 .../UnsafeRowComparatorForNormalDims.java       |  59 +++
 .../sort/unsafe/holder/SortTempChunkHolder.java |   3 +-
 .../holder/UnsafeFinalMergePageHolder.java      |  19 +-
 .../unsafe/holder/UnsafeInmemoryHolder.java     |  21 +-
 .../holder/UnsafeSortTempFileChunkHolder.java   | 138 ++----
 .../merger/UnsafeIntermediateFileMerger.java    | 118 +----
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  27 +-
 .../merger/CompactionResultSortProcessor.java   |   1 -
 .../sort/sortdata/IntermediateFileMerger.java   |  95 +---
 .../IntermediateSortTempRowComparator.java      |  73 +++
 .../sort/sortdata/NewRowComparator.java         |   5 +-
 .../sortdata/NewRowComparatorForNormalDims.java |   3 +-
 .../processing/sort/sortdata/RowComparator.java |  94 ----
 .../sortdata/RowComparatorForNormalDims.java    |  62 ---
 .../SingleThreadFinalSortFilesMerger.java       |  25 +-
 .../processing/sort/sortdata/SortDataRows.java  |  85 +---
 .../sort/sortdata/SortTempFileChunkHolder.java  | 174 ++-----
 .../sort/sortdata/TableFieldStat.java           | 176 +++++++
 28 files changed, 1186 insertions(+), 1294 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
index d6ecfbc..fca1244 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/NonDictionaryUtil.java
@@ -82,18 +82,26 @@ public class NonDictionaryUtil {
   }
 
   /**
-   * Method to get the required Dimension from obj []
+   * Method to get the required dictionary Dimension from obj []
    *
    * @param index
    * @param row
    * @return
    */
-  public static Integer getDimension(int index, Object[] row) {
-
-    Integer[] dimensions = (Integer[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
-
+  public static int getDictDimension(int index, Object[] row) {
+    int[] dimensions = (int[]) row[WriteStepRowUtil.DICTIONARY_DIMENSION];
     return dimensions[index];
+  }
 
+  /**
+   * Method to get the required non-dictionary & complex from 3-parted row
+   * @param index
+   * @param row
+   * @return
+   */
+  public static byte[] getNoDictOrComplex(int index, Object[] row) {
+    byte[][] nonDictArray = (byte[][]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
+    return nonDictArray[index];
   }
 
   /**
@@ -108,60 +116,11 @@ public class NonDictionaryUtil {
     return measures[index];
   }
 
-  public static byte[] getByteArrayForNoDictionaryCols(Object[] row) {
-
-    return (byte[]) row[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX];
-  }
-
   public static void prepareOutObj(Object[] out, int[] dimArray, byte[][] byteBufferArr,
       Object[] measureArray) {
-
     out[WriteStepRowUtil.DICTIONARY_DIMENSION] = dimArray;
     out[WriteStepRowUtil.NO_DICTIONARY_AND_COMPLEX] = byteBufferArr;
     out[WriteStepRowUtil.MEASURE] = measureArray;
 
   }
-
-  /**
-   * This method will extract the single dimension from the complete high card dims byte[].+     *
-   * The format of the byte [] will be,  Totallength,CompleteStartOffsets,Dat
-   *
-   * @param highCardArr
-   * @param index
-   * @param highCardinalityCount
-   * @param outBuffer
-   */
-  public static void extractSingleHighCardDims(byte[] highCardArr, int index,
-      int highCardinalityCount, ByteBuffer outBuffer) {
-    ByteBuffer buff = null;
-    short secIndex = 0;
-    short firstIndex = 0;
-    int length;
-    // if the requested index is a last one then we need to calculate length
-    // based on byte[] length.
-    if (index == highCardinalityCount - 1) {
-      // need to read 2 bytes(1 short) to determine starting offset and
-      // length can be calculated by array length.
-      buff = ByteBuffer.wrap(highCardArr, (index * 2) + 2, 2);
-    } else {
-      // need to read 4 bytes(2 short) to determine starting offset and
-      // length.
-      buff = ByteBuffer.wrap(highCardArr, (index * 2) + 2, 4);
-    }
-
-    firstIndex = buff.getShort();
-    // if it is a last dimension in high card then this will be last
-    // offset.so calculate length from total length
-    if (index == highCardinalityCount - 1) {
-      secIndex = (short) highCardArr.length;
-    } else {
-      secIndex = buff.getShort();
-    }
-
-    length = secIndex - firstIndex;
-
-    outBuffer.position(firstIndex);
-    outBuffer.limit(outBuffer.position() + length);
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index 1d7c791..7203278 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -383,7 +383,6 @@ object CarbonDataStoreCreator {
       .getInstance.createCache(CacheType.REVERSE_DICTIONARY)
 
     for (i <- set.indices) {
-      //      val dim = getDimension(dims, i).get
       val columnIdentifier: ColumnIdentifier =
         new ColumnIdentifier(dims.get(i).getColumnId, null, null)
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
index 5124247..0422239 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
@@ -35,7 +35,7 @@ import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.loading.parser.impl.RowParserImpl
-import org.apache.carbondata.processing.loading.sort.SortStepRowUtil
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler
 import org.apache.carbondata.processing.loading.steps.DataWriterProcessorStepImpl
 import org.apache.carbondata.processing.sort.sortdata.SortParameters
 import org.apache.carbondata.processing.store.{CarbonFactHandler, CarbonFactHandlerFactory}
@@ -206,7 +206,7 @@ object DataLoadProcessorStepOnSpark {
     val model: CarbonLoadModel = modelBroadcast.value.getCopyWithTaskNo(index.toString)
     val conf = DataLoadProcessBuilder.createConfiguration(model)
     val sortParameters = SortParameters.createSortParameters(conf)
-    val sortStepRowUtil = new SortStepRowUtil(sortParameters)
+    val sortStepRowHandler = new SortStepRowHandler(sortParameters)
     TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) =>
       wrapException(e, model)
     }
@@ -216,7 +216,7 @@ object DataLoadProcessorStepOnSpark {
 
       override def next(): CarbonRow = {
         val row =
-          new CarbonRow(sortStepRowUtil.convertRow(rows.next().getData))
+          new CarbonRow(sortStepRowHandler.convertRawRowTo3Parts(rows.next().getData))
         rowCounter.add(1)
         row
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java b/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
new file mode 100644
index 0000000..8d351cf
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/row/IntermediateSortTempRow.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.row;
+
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.DataTypeUtil;
+
+/**
+ * During sort procedure, each row will be written to sort temp file in this logic format.
+ * an intermediate sort temp row consists 3 parts:
+ * dictSort, noDictSort, noSortDimsAndMeasures(dictNoSort, noDictNoSort, measure)
+ */
+public class IntermediateSortTempRow {
+  private int[] dictSortDims;
+  private byte[][] noDictSortDims;
+  private byte[] noSortDimsAndMeasures;
+
+  public IntermediateSortTempRow(int[] dictSortDims, byte[][] noDictSortDims,
+      byte[] noSortDimsAndMeasures) {
+    this.dictSortDims = dictSortDims;
+    this.noDictSortDims = noDictSortDims;
+    this.noSortDimsAndMeasures = noSortDimsAndMeasures;
+  }
+
+  public int[] getDictSortDims() {
+    return dictSortDims;
+  }
+
+  public byte[][] getNoDictSortDims() {
+    return noDictSortDims;
+  }
+
+  public byte[] getNoSortDimsAndMeasures() {
+    return noSortDimsAndMeasures;
+  }
+
+  /**
+   * deserialize from bytes array to get the no sort fields
+   * @param outDictNoSort stores the dict & no-sort fields
+   * @param outNoDictNoSort stores the no-dict & no-sort fields, including complex
+   * @param outMeasures stores the measure fields
+   * @param dataTypes data type for the measure
+   */
+  public void unpackNoSortFromBytes(int[] outDictNoSort, byte[][] outNoDictNoSort,
+      Object[] outMeasures, DataType[] dataTypes) {
+    ByteBuffer rowBuffer = ByteBuffer.wrap(noSortDimsAndMeasures);
+    // read dict_no_sort
+    int dictNoSortCnt = outDictNoSort.length;
+    for (int i = 0; i < dictNoSortCnt; i++) {
+      outDictNoSort[i] = rowBuffer.getInt();
+    }
+
+    // read no_dict_no_sort (including complex)
+    int noDictNoSortCnt = outNoDictNoSort.length;
+    for (int i = 0; i < noDictNoSortCnt; i++) {
+      short len = rowBuffer.getShort();
+      byte[] bytes = new byte[len];
+      rowBuffer.get(bytes);
+      outNoDictNoSort[i] = bytes;
+    }
+
+    // read measure
+    int measureCnt = outMeasures.length;
+    DataType tmpDataType;
+    Object tmpContent;
+    for (short idx = 0 ; idx < measureCnt; idx++) {
+      if ((byte) 0 == rowBuffer.get()) {
+        outMeasures[idx] = null;
+        continue;
+      }
+
+      tmpDataType = dataTypes[idx];
+      if (DataTypes.BOOLEAN == tmpDataType) {
+        if ((byte) 1 == rowBuffer.get()) {
+          tmpContent = true;
+        } else {
+          tmpContent = false;
+        }
+      } else if (DataTypes.SHORT == tmpDataType) {
+        tmpContent = rowBuffer.getShort();
+      } else if (DataTypes.INT == tmpDataType) {
+        tmpContent = rowBuffer.getInt();
+      } else if (DataTypes.LONG == tmpDataType) {
+        tmpContent = rowBuffer.getLong();
+      } else if (DataTypes.DOUBLE == tmpDataType) {
+        tmpContent = rowBuffer.getDouble();
+      } else if (DataTypes.isDecimal(tmpDataType)) {
+        short len = rowBuffer.getShort();
+        byte[] decimalBytes = new byte[len];
+        rowBuffer.get(decimalBytes);
+        tmpContent = DataTypeUtil.byteToBigDecimal(decimalBytes);
+      } else {
+        throw new IllegalArgumentException("Unsupported data type: " + tmpDataType);
+      }
+      outMeasures[idx] = tmpContent;
+    }
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
new file mode 100644
index 0000000..f31a2b9
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowHandler.java
@@ -0,0 +1,466 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.apache.carbondata.core.memory.CarbonUnsafe;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
+
+/**
+ * This class is used to convert/write/read row in sort step in carbondata.
+ * It consists the following function:
+ * 1. convert raw row & intermediate sort temp row to 3-parted row
+ * 2. read/write intermediate sort temp row to sort temp file & unsafe memory
+ * 3. write raw row directly to sort temp file & unsafe memory as intermediate sort temp row
+ */
+public class SortStepRowHandler implements Serializable {
+  private static final long serialVersionUID = 1L;
+  private int dictSortDimCnt = 0;
+  private int dictNoSortDimCnt = 0;
+  private int noDictSortDimCnt = 0;
+  private int noDictNoSortDimCnt = 0;
+  private int measureCnt;
+
+  // indices for dict & sort dimension columns
+  private int[] dictSortDimIdx;
+  // indices for dict & no-sort dimension columns
+  private int[] dictNoSortDimIdx;
+  // indices for no-dict & sort dimension columns
+  private int[] noDictSortDimIdx;
+  // indices for no-dict & no-sort dimension columns, including complex columns
+  private int[] noDictNoSortDimIdx;
+  // indices for measure columns
+  private int[] measureIdx;
+
+  private DataType[] dataTypes;
+
+  /**
+   * constructor
+   * @param tableFieldStat table field stat
+   */
+  public SortStepRowHandler(TableFieldStat tableFieldStat) {
+    this.dictSortDimCnt = tableFieldStat.getDictSortDimCnt();
+    this.dictNoSortDimCnt = tableFieldStat.getDictNoSortDimCnt();
+    this.noDictSortDimCnt = tableFieldStat.getNoDictSortDimCnt();
+    this.noDictNoSortDimCnt = tableFieldStat.getNoDictNoSortDimCnt();
+    this.measureCnt = tableFieldStat.getMeasureCnt();
+    this.dictSortDimIdx = tableFieldStat.getDictSortDimIdx();
+    this.dictNoSortDimIdx = tableFieldStat.getDictNoSortDimIdx();
+    this.noDictSortDimIdx = tableFieldStat.getNoDictSortDimIdx();
+    this.noDictNoSortDimIdx = tableFieldStat.getNoDictNoSortDimIdx();
+    this.measureIdx = tableFieldStat.getMeasureIdx();
+    this.dataTypes = tableFieldStat.getMeasureDataType();
+  }
+
+  /**
+   * constructor
+   * @param sortParameters sort parameters
+   */
+  public SortStepRowHandler(SortParameters sortParameters) {
+    this(new TableFieldStat(sortParameters));
+  }
+
+  /**
+   * Convert carbon row from raw format to 3-parted format.
+   * This method is used in global-sort.
+   *
+   * @param row raw row whose length is the same as field number
+   * @return 3-parted row whose length is 3. (1 for dict dims ,1 for non-dict and complex,
+   * 1 for measures)
+   */
+  public Object[] convertRawRowTo3Parts(Object[] row) {
+    Object[] holder = new Object[3];
+    try {
+      int[] dictDims
+          = new int[this.dictSortDimCnt + this.dictNoSortDimCnt];
+      byte[][] nonDictArray = new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt][];
+      Object[] measures = new Object[this.measureCnt];
+
+      // convert dict & data
+      int idxAcc = 0;
+      for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+        dictDims[idxAcc++] = (int) row[this.dictSortDimIdx[idx]];
+      }
+
+      // convert dict & no-sort
+      for (int idx = 0; idx < this.dictNoSortDimCnt; idx++) {
+        dictDims[idxAcc++] = (int) row[this.dictNoSortDimIdx[idx]];
+      }
+      // convert no-dict & sort
+      idxAcc = 0;
+      for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+        nonDictArray[idxAcc++] = (byte[]) row[this.noDictSortDimIdx[idx]];
+      }
+      // convert no-dict & no-sort
+      for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) {
+        nonDictArray[idxAcc++] = (byte[]) row[this.noDictNoSortDimIdx[idx]];
+      }
+
+      // convert measure data
+      for (int idx = 0; idx < this.measureCnt; idx++) {
+        measures[idx] = row[this.measureIdx[idx]];
+      }
+
+      NonDictionaryUtil.prepareOutObj(holder, dictDims, nonDictArray, measures);
+    } catch (Exception e) {
+      throw new RuntimeException("Problem while converting row to 3 parts", e);
+    }
+    return holder;
+  }
+
+  /**
+   * Convert intermediate sort temp row to 3-parted row.
+   * This method is used in the final merge sort to feed rows to the next write step.
+   *
+   * @param sortTempRow intermediate sort temp row
+   * @return 3-parted row
+   */
+  public Object[] convertIntermediateSortTempRowTo3Parted(IntermediateSortTempRow sortTempRow) {
+    int[] dictDims
+        = new int[this.dictSortDimCnt + this.dictNoSortDimCnt];
+    byte[][] noDictArray
+        = new byte[this.noDictSortDimCnt + this.noDictNoSortDimCnt][];
+
+    int[] dictNoSortDims = new int[this.dictNoSortDimCnt];
+    byte[][] noDictNoSortDims = new byte[this.noDictNoSortDimCnt][];
+    Object[] measures = new Object[this.measureCnt];
+
+    sortTempRow.unpackNoSortFromBytes(dictNoSortDims, noDictNoSortDims, measures, this.dataTypes);
+
+    // dict dims
+    System.arraycopy(sortTempRow.getDictSortDims(), 0 , dictDims,
+        0, this.dictSortDimCnt);
+    System.arraycopy(dictNoSortDims, 0, dictDims,
+        this.dictSortDimCnt, this.dictNoSortDimCnt);;
+
+    // no dict dims, including complex
+    System.arraycopy(sortTempRow.getNoDictSortDims(), 0,
+        noDictArray, 0, this.noDictSortDimCnt);
+    System.arraycopy(noDictNoSortDims, 0, noDictArray,
+        this.noDictSortDimCnt, this.noDictNoSortDimCnt);
+
+    // measures are already here
+
+    Object[] holder = new Object[3];
+    NonDictionaryUtil.prepareOutObj(holder, dictDims, noDictArray, measures);
+    return holder;
+  }
+
+  /**
+   * Read intermediate sort temp row from InputStream.
+   * This method is used during the merge sort phase to read row from sort temp file.
+   *
+   * @param inputStream input stream
+   * @return a row that contains three parts
+   * @throws IOException if error occrus while reading from stream
+   */
+  public IntermediateSortTempRow readIntermediateSortTempRowFromInputStream(
+      DataInputStream inputStream) throws IOException {
+    int[] dictSortDims = new int[this.dictSortDimCnt];
+    byte[][] noDictSortDims = new byte[this.noDictSortDimCnt][];
+
+    // read dict & sort dim data
+    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+      dictSortDims[idx] = inputStream.readInt();
+    }
+
+    // read no-dict & sort data
+    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+      short len = inputStream.readShort();
+      byte[] bytes = new byte[len];
+      inputStream.readFully(bytes);
+      noDictSortDims[idx] = bytes;
+    }
+
+    // read no-dict dims & measures
+    int len = inputStream.readInt();
+    byte[] noSortDimsAndMeasures = new byte[len];
+    inputStream.readFully(noSortDimsAndMeasures);
+
+    return new IntermediateSortTempRow(dictSortDims, noDictSortDims, noSortDimsAndMeasures);
+  }
+
+  /**
+   * Write intermediate sort temp row to OutputStream
+   * This method is used during the merge sort phase to write row to sort temp file.
+   *
+   * @param sortTempRow intermediate sort temp row
+   * @param outputStream output stream
+   * @throws IOException if error occurs while writing to stream
+   */
+  public void writeIntermediateSortTempRowToOutputStream(IntermediateSortTempRow sortTempRow,
+      DataOutputStream outputStream) throws IOException {
+    // write dict & sort dim
+    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+      outputStream.writeInt(sortTempRow.getDictSortDims()[idx]);
+    }
+
+    // write no-dict & sort dim
+    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+      byte[] bytes = sortTempRow.getNoDictSortDims()[idx];
+      outputStream.writeShort(bytes.length);
+      outputStream.write(bytes);
+    }
+
+    // write packed no-sort dim & measure
+    outputStream.writeInt(sortTempRow.getNoSortDimsAndMeasures().length);
+    outputStream.write(sortTempRow.getNoSortDimsAndMeasures());
+  }
+
+  /**
+   * Write raw row as an intermediate sort temp row to sort temp file.
+   * This method is used in the beginning of the sort phase. Comparing with converting raw row to
+   * intermediate sort temp row and then writing the converted one, Writing raw row directly will
+   * save the intermediate trivial loss.
+   * This method use an array backend buffer to save memory allocation. The buffer will be reused
+   * for all rows (per thread).
+   *
+   * @param row raw row
+   * @param outputStream output stream
+   * @param rowBuffer array backend buffer
+   * @throws IOException if error occurs while writing to stream
+   */
+  public void writeRawRowAsIntermediateSortTempRowToOutputStream(Object[] row,
+      DataOutputStream outputStream, ByteBuffer rowBuffer) throws IOException {
+    // write dict & sort
+    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+      outputStream.writeInt((int) row[this.dictSortDimIdx[idx]]);
+    }
+
+    // write no-dict & sort
+    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+      byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]];
+      outputStream.writeShort(bytes.length);
+      outputStream.write(bytes);
+    }
+
+    // pack no-sort
+    rowBuffer.clear();
+    packNoSortFieldsToBytes(row, rowBuffer);
+    rowBuffer.flip();
+    int packSize = rowBuffer.limit();
+
+    // write no-sort
+    outputStream.writeInt(packSize);
+    outputStream.write(rowBuffer.array(), 0, packSize);
+  }
+
+  /**
+   * Read intermediate sort temp row from unsafe memory.
+   * This method is used during merge sort phase for off-heap sort.
+   *
+   * @param baseObject base object of memory block
+   * @param address address of the row
+   * @return intermediate sort temp row
+   */
+  public IntermediateSortTempRow readIntermediateSortTempRowFromUnsafeMemory(Object baseObject,
+      long address) {
+    int size = 0;
+
+    int[] dictSortDims = new int[this.dictSortDimCnt];
+    byte[][] noDictSortDims = new byte[this.noDictSortDimCnt][];
+
+    // read dict & sort dim
+    for (int idx = 0; idx < dictSortDims.length; idx++) {
+      dictSortDims[idx] = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+      size += 4;
+    }
+
+    // read no-dict & sort dim
+    for (int idx = 0; idx < noDictSortDims.length; idx++) {
+      short length = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+      size += 2;
+      byte[] bytes = new byte[length];
+      CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
+          bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length);
+      size += length;
+      noDictSortDims[idx] = bytes;
+    }
+
+    // read no-sort dims & measures
+    int len = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+    size += 4;
+    byte[] noSortDimsAndMeasures = new byte[len];
+    CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
+        noSortDimsAndMeasures, CarbonUnsafe.BYTE_ARRAY_OFFSET, len);
+
+    return new IntermediateSortTempRow(dictSortDims, noDictSortDims, noSortDimsAndMeasures);
+  }
+
+  /**
+   * Write intermediate sort temp row directly from unsafe memory to stream.
+   * This method is used at the late beginning of the sort phase to write in-memory pages
+   * to sort temp file. Comparing with reading intermediate sort temp row from memory and then
+   * writing it, Writing directly from memory to stream will save the intermediate trivial loss.
+   *
+   * @param baseObject base object of the memory block
+   * @param address base address of the row
+   * @param outputStream output stream
+   * @throws IOException if error occurs while writing to stream
+   */
+  public void writeIntermediateSortTempRowFromUnsafeMemoryToStream(Object baseObject,
+      long address, DataOutputStream outputStream) throws IOException {
+    int size = 0;
+
+    // dict & sort
+    for (int idx = 0; idx < dictSortDimCnt; idx++) {
+      outputStream.writeInt(CarbonUnsafe.getUnsafe().getInt(baseObject, address + size));
+      size += 4;
+    }
+
+    // no-dict & sort
+    for (int idx = 0; idx < noDictSortDimCnt; idx++) {
+      short length = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+      size += 2;
+      byte[] bytes = new byte[length];
+      CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
+          bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, length);
+      size += length;
+
+      outputStream.writeShort(length);
+      outputStream.write(bytes);
+    }
+
+    // packed no-sort & measure
+    int len = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+    size += 4;
+    byte[] noSortDimsAndMeasures = new byte[len];
+    CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size,
+        noSortDimsAndMeasures, CarbonUnsafe.BYTE_ARRAY_OFFSET, len);
+    size += len;
+
+    outputStream.writeInt(len);
+    outputStream.write(noSortDimsAndMeasures);
+  }
+
+  /**
+   * Write raw row as an intermediate sort temp row to memory.
+   * This method is used in the beginning of the off-heap sort phase. Comparing with converting
+   * raw row to intermediate sort temp row and then writing the converted one,
+   * Writing raw row directly will save the intermediate trivial loss.
+   * This method use an array backend buffer to save memory allocation. The buffer will be reused
+   * for all rows (per thread).
+   *
+   * @param row raw row
+   * @param baseObject base object of the memory block
+   * @param address base address for the row
+   * @param rowBuffer array backend buffer
+   * @return number of bytes written to memory
+   */
+  public int writeRawRowAsIntermediateSortTempRowToUnsafeMemory(Object[] row,
+      Object baseObject, long address, ByteBuffer rowBuffer) {
+    int size = 0;
+    // write dict & sort
+    for (int idx = 0; idx < this.dictSortDimCnt; idx++) {
+      CarbonUnsafe.getUnsafe()
+          .putInt(baseObject, address + size, (int) row[this.dictSortDimIdx[idx]]);
+      size += 4;
+    }
+
+    // write no-dict & sort
+    for (int idx = 0; idx < this.noDictSortDimCnt; idx++) {
+      byte[] bytes = (byte[]) row[this.noDictSortDimIdx[idx]];
+      CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) bytes.length);
+      size += 2;
+      CarbonUnsafe.getUnsafe()
+          .copyMemory(bytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, address + size,
+              bytes.length);
+      size += bytes.length;
+    }
+
+    // convert pack no-sort
+    rowBuffer.clear();
+    packNoSortFieldsToBytes(row, rowBuffer);
+    rowBuffer.flip();
+    int packSize = rowBuffer.limit();
+
+    // write no-sort
+    CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, packSize);
+    size += 4;
+    CarbonUnsafe.getUnsafe()
+        .copyMemory(rowBuffer.array(), CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject, address + size,
+            packSize);
+    size += packSize;
+    return size;
+  }
+
+  /**
+   * Pack to no-sort fields to byte array
+   *
+   * @param row raw row
+   * @param rowBuffer byte array backend buffer
+   */
+  private void packNoSortFieldsToBytes(Object[] row, ByteBuffer rowBuffer) {
+    // convert dict & no-sort
+    for (int idx = 0; idx < this.dictNoSortDimCnt; idx++) {
+      rowBuffer.putInt((int) row[this.dictNoSortDimIdx[idx]]);
+    }
+    // convert no-dict & no-sort
+    for (int idx = 0; idx < this.noDictNoSortDimCnt; idx++) {
+      byte[] bytes = (byte[]) row[this.noDictNoSortDimIdx[idx]];
+      rowBuffer.putShort((short) bytes.length);
+      rowBuffer.put(bytes);
+    }
+
+    // convert measure
+    Object tmpValue;
+    DataType tmpDataType;
+    for (int idx = 0; idx < this.measureCnt; idx++) {
+      tmpValue = row[this.measureIdx[idx]];
+      tmpDataType = this.dataTypes[idx];
+      if (null == tmpValue) {
+        rowBuffer.put((byte) 0);
+        continue;
+      }
+      rowBuffer.put((byte) 1);
+      if (DataTypes.BOOLEAN == tmpDataType) {
+        if ((boolean) tmpValue) {
+          rowBuffer.put((byte) 1);
+        } else {
+          rowBuffer.put((byte) 0);
+        }
+      } else if (DataTypes.SHORT == tmpDataType) {
+        rowBuffer.putShort((Short) tmpValue);
+      } else if (DataTypes.INT == tmpDataType) {
+        rowBuffer.putInt((Integer) tmpValue);
+      } else if (DataTypes.LONG == tmpDataType) {
+        rowBuffer.putLong((Long) tmpValue);
+      } else if (DataTypes.DOUBLE == tmpDataType) {
+        rowBuffer.putDouble((Double) tmpValue);
+      } else if (DataTypes.isDecimal(tmpDataType)) {
+        byte[] decimalBytes = DataTypeUtil.bigDecimalToByte((BigDecimal) tmpValue);
+        rowBuffer.putShort((short) decimalBytes.length);
+        rowBuffer.put(decimalBytes);
+      } else {
+        throw new IllegalArgumentException("Unsupported data type: " + tmpDataType);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
deleted file mode 100644
index c4e4756..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.sort;
-
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-import org.apache.carbondata.processing.sort.sortdata.SortParameters;
-
-public class SortStepRowUtil {
-  private int measureCount;
-  private int dimensionCount;
-  private int complexDimensionCount;
-  private int noDictionaryCount;
-  private int[] dictDimIdx;
-  private int[] nonDictIdx;
-  private int[] measureIdx;
-
-  public SortStepRowUtil(SortParameters parameters) {
-    this.measureCount = parameters.getMeasureColCount();
-    this.dimensionCount = parameters.getDimColCount();
-    this.complexDimensionCount = parameters.getComplexDimColCount();
-    this.noDictionaryCount = parameters.getNoDictionaryCount();
-    boolean[] isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
-
-    int index = 0;
-    int nonDicIndex = 0;
-    int allCount = 0;
-
-    // be careful that the default value is 0
-    this.dictDimIdx = new int[dimensionCount - noDictionaryCount];
-    this.nonDictIdx = new int[noDictionaryCount + complexDimensionCount];
-    this.measureIdx = new int[measureCount];
-
-    // indices for dict dim columns
-    for (int i = 0; i < isNoDictionaryDimensionColumn.length; i++) {
-      if (isNoDictionaryDimensionColumn[i]) {
-        nonDictIdx[nonDicIndex++] = i;
-      } else {
-        dictDimIdx[index++] = allCount;
-      }
-      allCount++;
-    }
-
-    // indices for non dict dim/complex columns
-    for (int i = 0; i < complexDimensionCount; i++) {
-      nonDictIdx[nonDicIndex++] = allCount;
-      allCount++;
-    }
-
-    // indices for measure columns
-    for (int i = 0; i < measureCount; i++) {
-      measureIdx[i] = allCount;
-      allCount++;
-    }
-  }
-
-  public Object[] convertRow(Object[] data) {
-    // create new row of size 3 (1 for dims , 1 for high card , 1 for measures)
-    Object[] holder = new Object[3];
-    try {
-
-      int[] dictDims = new int[dimensionCount - noDictionaryCount];
-      byte[][] nonDictArray = new byte[noDictionaryCount + complexDimensionCount][];
-      Object[] measures = new Object[measureCount];
-
-      // write dict dim data
-      for (int idx = 0; idx < dictDimIdx.length; idx++) {
-        dictDims[idx] = (int) data[dictDimIdx[idx]];
-      }
-
-      // write non dict dim data
-      for (int idx = 0; idx < nonDictIdx.length; idx++) {
-        nonDictArray[idx] = (byte[]) data[nonDictIdx[idx]];
-      }
-
-      // write measure data
-      for (int idx = 0; idx < measureIdx.length; idx++) {
-        measures[idx] = data[measureIdx[idx]];
-      }
-      NonDictionaryUtil.prepareOutObj(holder, dictDims, nonDictArray, measures);
-
-      // increment number if record read
-    } catch (Exception e) {
-      throw new RuntimeException("Problem while converting row ", e);
-    }
-    //return out row
-    return holder;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
index e5583c2..7ea5cb3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
@@ -19,35 +19,20 @@ package org.apache.carbondata.processing.loading.sort.unsafe;
 
 import java.io.DataOutputStream;
 import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.Arrays;
+import java.nio.ByteBuffer;
 
-import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.memory.IntPointerBuffer;
 import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.memory.UnsafeSortMemoryManager;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.core.util.DataTypeUtil;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
+import org.apache.carbondata.processing.loading.sort.SortStepRowHandler;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 /**
  * It can keep the data of prescribed size data in offheap/onheap memory and returns it when needed
  */
 public class UnsafeCarbonRowPage {
-
-  private boolean[] noDictionaryDimensionMapping;
-
-  private boolean[] noDictionarySortColumnMapping;
-
-  private int dimensionSize;
-
-  private int measureSize;
-
-  private DataType[] measureDataType;
-
-  private long[] nullSetWords;
-
   private IntPointerBuffer buffer;
 
   private int lastSize;
@@ -62,16 +47,14 @@ public class UnsafeCarbonRowPage {
 
   private long taskId;
 
-  public UnsafeCarbonRowPage(boolean[] noDictionaryDimensionMapping,
-      boolean[] noDictionarySortColumnMapping, int dimensionSize, int measureSize, DataType[] type,
-      MemoryBlock memoryBlock, boolean saveToDisk, long taskId) {
-    this.noDictionaryDimensionMapping = noDictionaryDimensionMapping;
-    this.noDictionarySortColumnMapping = noDictionarySortColumnMapping;
-    this.dimensionSize = dimensionSize;
-    this.measureSize = measureSize;
-    this.measureDataType = type;
+  private TableFieldStat tableFieldStat;
+  private SortStepRowHandler sortStepRowHandler;
+
+  public UnsafeCarbonRowPage(TableFieldStat tableFieldStat, MemoryBlock memoryBlock,
+      boolean saveToDisk, long taskId) {
+    this.tableFieldStat = tableFieldStat;
+    this.sortStepRowHandler = new SortStepRowHandler(tableFieldStat);
     this.saveToDisk = saveToDisk;
-    this.nullSetWords = new long[((measureSize - 1) >> 6) + 1];
     this.taskId = taskId;
     buffer = new IntPointerBuffer(this.taskId);
     this.dataBlock = memoryBlock;
@@ -80,255 +63,44 @@ public class UnsafeCarbonRowPage {
     this.managerType = MemoryManagerType.UNSAFE_MEMORY_MANAGER;
   }
 
-  public int addRow(Object[] row) {
-    int size = addRow(row, dataBlock.getBaseOffset() + lastSize);
+  public int addRow(Object[] row, ByteBuffer rowBuffer) {
+    int size = addRow(row, dataBlock.getBaseOffset() + lastSize, rowBuffer);
     buffer.set(lastSize);
     lastSize = lastSize + size;
     return size;
   }
 
-  private int addRow(Object[] row, long address) {
-    if (row == null) {
-      throw new RuntimeException("Row is null ??");
-    }
-    int dimCount = 0;
-    int size = 0;
-    Object baseObject = dataBlock.getBaseObject();
-    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
-      if (noDictionaryDimensionMapping[dimCount]) {
-        byte[] col = (byte[]) row[dimCount];
-        CarbonUnsafe.getUnsafe()
-            .putShort(baseObject, address + size, (short) col.length);
-        size += 2;
-        CarbonUnsafe.getUnsafe().copyMemory(col, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
-            address + size, col.length);
-        size += col.length;
-      } else {
-        int value = (int) row[dimCount];
-        CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, value);
-        size += 4;
-      }
-    }
-
-    // write complex dimensions here.
-    for (; dimCount < dimensionSize; dimCount++) {
-      byte[] col = (byte[]) row[dimCount];
-      CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, (short) col.length);
-      size += 2;
-      CarbonUnsafe.getUnsafe().copyMemory(col, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
-          address + size, col.length);
-      size += col.length;
-    }
-    Arrays.fill(nullSetWords, 0);
-    int nullSetSize = nullSetWords.length * 8;
-    int nullWordLoc = size;
-    size += nullSetSize;
-    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
-      Object value = row[mesCount + dimensionSize];
-      if (null != value) {
-        DataType dataType = measureDataType[mesCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          Boolean bval = (Boolean) value;
-          CarbonUnsafe.getUnsafe().putBoolean(baseObject, address + size, bval);
-          size += 1;
-        } else if (dataType == DataTypes.SHORT) {
-          Short sval = (Short) value;
-          CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, sval);
-          size += 2;
-        } else if (dataType == DataTypes.INT) {
-          Integer ival = (Integer) value;
-          CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, ival);
-          size += 4;
-        } else if (dataType == DataTypes.LONG) {
-          Long val = (Long) value;
-          CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, val);
-          size += 8;
-        } else if (dataType == DataTypes.DOUBLE) {
-          Double doubleVal = (Double) value;
-          CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, doubleVal);
-          size += 8;
-        } else if (DataTypes.isDecimal(dataType)) {
-          BigDecimal decimalVal = (BigDecimal) value;
-          byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(decimalVal);
-          CarbonUnsafe.getUnsafe()
-              .putShort(baseObject, address + size, (short) bigDecimalInBytes.length);
-          size += 2;
-          CarbonUnsafe.getUnsafe()
-              .copyMemory(bigDecimalInBytes, CarbonUnsafe.BYTE_ARRAY_OFFSET, baseObject,
-                  address + size, bigDecimalInBytes.length);
-          size += bigDecimalInBytes.length;
-        } else {
-          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
-        }
-        set(nullSetWords, mesCount);
-      } else {
-        unset(nullSetWords, mesCount);
-      }
-    }
-    CarbonUnsafe.getUnsafe().copyMemory(nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET, baseObject,
-        address + nullWordLoc, nullSetSize);
-    return size;
+  /**
+   * add raw row as intermidiate sort temp row to page
+   *
+   * @param row
+   * @param address
+   * @return
+   */
+  private int addRow(Object[] row, long address, ByteBuffer rowBuffer) {
+    return sortStepRowHandler.writeRawRowAsIntermediateSortTempRowToUnsafeMemory(row,
+        dataBlock.getBaseObject(), address, rowBuffer);
   }
 
-  public Object[] getRow(long address, Object[] rowToFill) {
-    int dimCount = 0;
-    int size = 0;
-
-    Object baseObject = dataBlock.getBaseObject();
-    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
-      if (noDictionaryDimensionMapping[dimCount]) {
-        short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-        byte[] col = new byte[aShort];
-        size += 2;
-        CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                col.length);
-        size += col.length;
-        rowToFill[dimCount] = col;
-      } else {
-        int anInt = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-        size += 4;
-        rowToFill[dimCount] = anInt;
-      }
-    }
-
-    // write complex dimensions here.
-    for (; dimCount < dimensionSize; dimCount++) {
-      short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-      byte[] col = new byte[aShort];
-      size += 2;
-      CarbonUnsafe.getUnsafe()
-          .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET, col.length);
-      size += col.length;
-      rowToFill[dimCount] = col;
-    }
-
-    int nullSetSize = nullSetWords.length * 8;
-    Arrays.fill(nullSetWords, 0);
-    CarbonUnsafe.getUnsafe()
-        .copyMemory(baseObject, address + size, nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET,
-            nullSetSize);
-    size += nullSetSize;
-
-    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
-      if (isSet(nullSetWords, mesCount)) {
-        DataType dataType = measureDataType[mesCount];
-        if (dataType == DataTypes.BOOLEAN) {
-          Boolean bval = CarbonUnsafe.getUnsafe().getBoolean(baseObject, address + size);
-          size += 1;
-          rowToFill[dimensionSize + mesCount] = bval;
-        } else if (dataType == DataTypes.SHORT) {
-          Short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-          size += 2;
-          rowToFill[dimensionSize + mesCount] = sval;
-        } else if (dataType == DataTypes.INT) {
-          Integer ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-          size += 4;
-          rowToFill[dimensionSize + mesCount] = ival;
-        } else if (dataType == DataTypes.LONG) {
-          Long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
-          size += 8;
-          rowToFill[dimensionSize + mesCount] = val;
-        } else if (dataType == DataTypes.DOUBLE) {
-          Double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
-          size += 8;
-          rowToFill[dimensionSize + mesCount] = doubleVal;
-        } else if (DataTypes.isDecimal(dataType)) {
-          short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-          byte[] bigDecimalInBytes = new byte[aShort];
-          size += 2;
-          CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
-              CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
-          size += bigDecimalInBytes.length;
-          rowToFill[dimensionSize + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
-        } else {
-          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
-        }
-      } else {
-        rowToFill[dimensionSize + mesCount] = null;
-      }
-    }
-    return rowToFill;
+  /**
+   * get one row from memory address
+   * @param address address
+   * @return one row
+   */
+  public IntermediateSortTempRow getRow(long address) {
+    return sortStepRowHandler.readIntermediateSortTempRowFromUnsafeMemory(
+        dataBlock.getBaseObject(), address);
   }
 
-  public void fillRow(long address, DataOutputStream stream) throws IOException {
-    int dimCount = 0;
-    int size = 0;
-
-    Object baseObject = dataBlock.getBaseObject();
-    for (; dimCount < noDictionaryDimensionMapping.length; dimCount++) {
-      if (noDictionaryDimensionMapping[dimCount]) {
-        short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-        byte[] col = new byte[aShort];
-        size += 2;
-        CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                col.length);
-        size += col.length;
-        stream.writeShort(aShort);
-        stream.write(col);
-      } else {
-        int anInt = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-        size += 4;
-        stream.writeInt(anInt);
-      }
-    }
-
-    // write complex dimensions here.
-    for (; dimCount < dimensionSize; dimCount++) {
-      short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-      byte[] col = new byte[aShort];
-      size += 2;
-      CarbonUnsafe.getUnsafe()
-          .copyMemory(baseObject, address + size, col, CarbonUnsafe.BYTE_ARRAY_OFFSET, col.length);
-      size += col.length;
-      stream.writeShort(aShort);
-      stream.write(col);
-    }
-
-    int nullSetSize = nullSetWords.length * 8;
-    Arrays.fill(nullSetWords, 0);
-    CarbonUnsafe.getUnsafe()
-        .copyMemory(baseObject, address + size, nullSetWords, CarbonUnsafe.LONG_ARRAY_OFFSET,
-            nullSetSize);
-    size += nullSetSize;
-    for (int i = 0; i < nullSetWords.length; i++) {
-      stream.writeLong(nullSetWords[i]);
-    }
-
-    for (int mesCount = 0; mesCount < measureSize; mesCount++) {
-      if (isSet(nullSetWords, mesCount)) {
-        DataType dataType = measureDataType[mesCount];
-        if (dataType == DataTypes.SHORT) {
-          short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-          size += 2;
-          stream.writeShort(sval);
-        } else if (dataType == DataTypes.INT) {
-          int ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-          size += 4;
-          stream.writeInt(ival);
-        } else if (dataType == DataTypes.LONG) {
-          long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
-          size += 8;
-          stream.writeLong(val);
-        } else if (dataType == DataTypes.DOUBLE) {
-          double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
-          size += 8;
-          stream.writeDouble(doubleVal);
-        } else if (DataTypes.isDecimal(dataType)) {
-          short aShort = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-          byte[] bigDecimalInBytes = new byte[aShort];
-          size += 2;
-          CarbonUnsafe.getUnsafe().copyMemory(baseObject, address + size, bigDecimalInBytes,
-              CarbonUnsafe.BYTE_ARRAY_OFFSET, bigDecimalInBytes.length);
-          size += bigDecimalInBytes.length;
-          stream.writeShort(aShort);
-          stream.write(bigDecimalInBytes);
-        } else {
-          throw new IllegalArgumentException("unsupported data type:" + measureDataType[mesCount]);
-        }
-      }
-    }
+  /**
+   * write a row to stream
+   * @param address address of a row
+   * @param stream stream
+   * @throws IOException
+   */
+  public void writeRow(long address, DataOutputStream stream) throws IOException {
+    sortStepRowHandler.writeIntermediateSortTempRowFromUnsafeMemoryToStream(
+        dataBlock.getBaseObject(), address, stream);
   }
 
   public void freeMemory() {
@@ -362,27 +134,8 @@ public class UnsafeCarbonRowPage {
     return dataBlock;
   }
 
-  public static void set(long[] words, int index) {
-    int wordOffset = (index >> 6);
-    words[wordOffset] |= (1L << index);
-  }
-
-  public static void unset(long[] words, int index) {
-    int wordOffset = (index >> 6);
-    words[wordOffset] &= ~(1L << index);
-  }
-
-  public static boolean isSet(long[] words, int index) {
-    int wordOffset = (index >> 6);
-    return ((words[wordOffset] & (1L << index)) != 0);
-  }
-
-  public boolean[] getNoDictionaryDimensionMapping() {
-    return noDictionaryDimensionMapping;
-  }
-
-  public boolean[] getNoDictionarySortColumnMapping() {
-    return noDictionarySortColumnMapping;
+  public TableFieldStat getTableFieldStat() {
+    return tableFieldStat;
   }
 
   public void setNewDataBlock(MemoryBlock newMemoryBlock) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
index 4dd5e44..5d038d3 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.processing.loading.sort.unsafe;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -41,13 +42,14 @@ import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
 import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparator;
-import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparatorForNormalDIms;
+import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparatorForNormalDims;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeIntermediateMerger;
 import org.apache.carbondata.processing.loading.sort.unsafe.sort.TimSort;
 import org.apache.carbondata.processing.loading.sort.unsafe.sort.UnsafeIntSortDataFormat;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 public class UnsafeSortDataRows {
@@ -69,7 +71,8 @@ public class UnsafeSortDataRows {
    */
 
   private SortParameters parameters;
-
+  private TableFieldStat tableFieldStat;
+  private ThreadLocal<ByteBuffer> rowBuffer;
   private UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger;
 
   private UnsafeCarbonRowPage rowPage;
@@ -94,7 +97,13 @@ public class UnsafeSortDataRows {
   public UnsafeSortDataRows(SortParameters parameters,
       UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger, int inMemoryChunkSize) {
     this.parameters = parameters;
-
+    this.tableFieldStat = new TableFieldStat(parameters);
+    this.rowBuffer = new ThreadLocal<ByteBuffer>() {
+      @Override protected ByteBuffer initialValue() {
+        byte[] backedArray = new byte[2 * 1024 * 1024];
+        return ByteBuffer.wrap(backedArray);
+      }
+    };
     this.unsafeInMemoryIntermediateFileMerger = unsafeInMemoryIntermediateFileMerger;
 
     // observer of writing file in thread
@@ -127,11 +136,7 @@ public class UnsafeSortDataRows {
     if (isMemoryAvailable) {
       UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(baseBlock.size());
     }
-    this.rowPage = new UnsafeCarbonRowPage(parameters.getNoDictionaryDimnesionColumn(),
-        parameters.getNoDictionarySortColumn(),
-        parameters.getDimColCount() + parameters.getComplexDimColCount(),
-        parameters.getMeasureColCount(), parameters.getMeasureDataType(), baseBlock,
-        !isMemoryAvailable, taskId);
+    this.rowPage = new UnsafeCarbonRowPage(tableFieldStat, baseBlock, !isMemoryAvailable, taskId);
     // Delete if any older file exists in sort temp folder
     deleteSortLocationIfExists();
 
@@ -178,7 +183,7 @@ public class UnsafeSortDataRows {
   private void addBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
     for (int i = 0; i < size; i++) {
       if (rowPage.canAdd()) {
-        bytesAdded += rowPage.addRow(rowBatch[i]);
+        bytesAdded += rowPage.addRow(rowBatch[i], rowBuffer.get());
       } else {
         try {
           if (enableInMemoryIntermediateMerge) {
@@ -194,15 +199,8 @@ public class UnsafeSortDataRows {
           if (!saveToDisk) {
             UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(memoryBlock.size());
           }
-          rowPage = new UnsafeCarbonRowPage(
-                  parameters.getNoDictionaryDimnesionColumn(),
-                  parameters.getNoDictionarySortColumn(),
-                  parameters.getDimColCount() + parameters.getComplexDimColCount(),
-                  parameters.getMeasureColCount(),
-                  parameters.getMeasureDataType(),
-                  memoryBlock,
-                  saveToDisk, taskId);
-          bytesAdded += rowPage.addRow(rowBatch[i]);
+          rowPage = new UnsafeCarbonRowPage(tableFieldStat, memoryBlock, saveToDisk, taskId);
+          bytesAdded += rowPage.addRow(rowBatch[i], rowBuffer.get());
         } catch (Exception e) {
           LOGGER.error(
                   "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
@@ -220,7 +218,7 @@ public class UnsafeSortDataRows {
     // if record holder list size is equal to sort buffer size then it will
     // sort the list and then write current list data to file
     if (rowPage.canAdd()) {
-      rowPage.addRow(row);
+      rowPage.addRow(row, rowBuffer.get());
     } else {
       try {
         if (enableInMemoryIntermediateMerge) {
@@ -235,13 +233,8 @@ public class UnsafeSortDataRows {
         if (!saveToDisk) {
           UnsafeSortMemoryManager.INSTANCE.allocateDummyMemory(memoryBlock.size());
         }
-        rowPage = new UnsafeCarbonRowPage(
-            parameters.getNoDictionaryDimnesionColumn(),
-            parameters.getNoDictionarySortColumn(),
-            parameters.getDimColCount(), parameters.getMeasureColCount(),
-            parameters.getMeasureDataType(), memoryBlock,
-            saveToDisk, taskId);
-        rowPage.addRow(row);
+        rowPage = new UnsafeCarbonRowPage(tableFieldStat, memoryBlock, saveToDisk, taskId);
+        rowPage.addRow(row, rowBuffer.get());
       } catch (Exception e) {
         LOGGER.error(
             "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
@@ -269,7 +262,7 @@ public class UnsafeSortDataRows {
             new UnsafeRowComparator(rowPage));
       } else {
         timSort.sort(rowPage.getBuffer(), 0, rowPage.getBuffer().getActualSize(),
-            new UnsafeRowComparatorForNormalDIms(rowPage));
+            new UnsafeRowComparatorForNormalDims(rowPage));
       }
       unsafeInMemoryIntermediateFileMerger.addDataChunkToMerge(rowPage);
     } else {
@@ -295,10 +288,9 @@ public class UnsafeSortDataRows {
       // write number of entries to the file
       stream.writeInt(actualSize);
       for (int i = 0; i < actualSize; i++) {
-        rowPage.fillRow(rowPage.getBuffer().get(i) + rowPage.getDataBlock().getBaseOffset(),
-            stream);
+        rowPage.writeRow(
+            rowPage.getBuffer().get(i) + rowPage.getDataBlock().getBaseOffset(), stream);
       }
-
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
     } finally {
@@ -367,7 +359,7 @@ public class UnsafeSortDataRows {
               new UnsafeRowComparator(page));
         } else {
           timSort.sort(page.getBuffer(), 0, page.getBuffer().getActualSize(),
-              new UnsafeRowComparatorForNormalDIms(page));
+              new UnsafeRowComparatorForNormalDims(page));
         }
         if (page.isSaveToDisk()) {
           // create a new file every time
@@ -380,7 +372,8 @@ public class UnsafeSortDataRows {
           writeDataToFile(page, sortTempFile);
           LOGGER.info("Time taken to sort row page with size" + page.getBuffer().getActualSize()
               + " and write is: " + (System.currentTimeMillis() - startTime) + ": location:"
-              + sortTempFile);
+              + sortTempFile + ", sort temp file size in MB is "
+              + sortTempFile.length() * 0.1 * 10 / 1024 / 1024);
           page.freeMemory();
           // add sort temp filename to and arrayList. When the list size reaches 20 then
           // intermediate merging of sort temp files will be triggered

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
index d02be9b..33342dc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
@@ -23,63 +23,25 @@ import org.apache.carbondata.core.memory.CarbonUnsafe;
 import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
+import org.apache.carbondata.processing.sort.sortdata.TableFieldStat;
 
 public class UnsafeRowComparator implements Comparator<UnsafeCarbonRow> {
-
-  /**
-   * mapping of dictionary and no dictionary of sort_columns.
-   */
-  private boolean[] noDictionarySortColumnMaping;
-
   private Object baseObject;
+  private TableFieldStat tableFieldStat;
+  private int dictSizeInMemory;
 
   public UnsafeRowComparator(UnsafeCarbonRowPage rowPage) {
-    this.noDictionarySortColumnMaping = rowPage.getNoDictionarySortColumnMapping();
     this.baseObject = rowPage.getDataBlock().getBaseObject();
+    this.tableFieldStat = rowPage.getTableFieldStat();
+    this.dictSizeInMemory = (tableFieldStat.getDictSortDimCnt()
+        + tableFieldStat.getDictNoSortDimCnt()) * 4;
   }
 
   /**
    * Below method will be used to compare two mdkey
    */
   public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
-    int diff = 0;
-    long rowA = rowL.address;
-    long rowB = rowR.address;
-    int sizeA = 0;
-    int sizeB = 0;
-    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
-      if (isNoDictionary) {
-        short aShort1 = CarbonUnsafe.getUnsafe().getShort(baseObject, rowA + sizeA);
-        byte[] byteArr1 = new byte[aShort1];
-        sizeA += 2;
-        CarbonUnsafe.getUnsafe().copyMemory(baseObject, rowA + sizeA, byteArr1,
-            CarbonUnsafe.BYTE_ARRAY_OFFSET, aShort1);
-        sizeA += aShort1;
-
-        short aShort2 = CarbonUnsafe.getUnsafe().getShort(baseObject, rowB + sizeB);
-        byte[] byteArr2 = new byte[aShort2];
-        sizeB += 2;
-        CarbonUnsafe.getUnsafe().copyMemory(baseObject, rowB + sizeB, byteArr2,
-            CarbonUnsafe.BYTE_ARRAY_OFFSET, aShort2);
-        sizeB += aShort2;
-
-        int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
-        if (difference != 0) {
-          return difference;
-        }
-      } else {
-        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
-        sizeA += 4;
-        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
-        sizeB += 4;
-        diff = dimFieldA - dimFieldB;
-        if (diff != 0) {
-          return diff;
-        }
-      }
-    }
-
-    return diff;
+    return compare(rowL, baseObject, rowR, baseObject);
   }
 
   /**
@@ -90,35 +52,40 @@ public class UnsafeRowComparator implements Comparator<UnsafeCarbonRow> {
     int diff = 0;
     long rowA = rowL.address;
     long rowB = rowR.address;
-    int sizeA = 0;
-    int sizeB = 0;
-    for (boolean isNoDictionary : noDictionarySortColumnMaping) {
+    int sizeInDictPartA = 0;
+
+    int sizeInNonDictPartA = 0;
+    int sizeInDictPartB = 0;
+    int sizeInNonDictPartB = 0;
+    for (boolean isNoDictionary : tableFieldStat.getIsSortColNoDictFlags()) {
       if (isNoDictionary) {
-        short aShort1 = CarbonUnsafe.getUnsafe().getShort(baseObjectL, rowA + sizeA);
-        byte[] byteArr1 = new byte[aShort1];
-        sizeA += 2;
+        short lengthA = CarbonUnsafe.getUnsafe().getShort(baseObjectL,
+            rowA + dictSizeInMemory + sizeInNonDictPartA);
+        byte[] byteArr1 = new byte[lengthA];
+        sizeInNonDictPartA += 2;
         CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObjectL, rowA + sizeA, byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                aShort1);
-        sizeA += aShort1;
+            .copyMemory(baseObjectL, rowA + dictSizeInMemory + sizeInNonDictPartA,
+                byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthA);
+        sizeInNonDictPartA += lengthA;
 
-        short aShort2 = CarbonUnsafe.getUnsafe().getShort(baseObjectR, rowB + sizeB);
-        byte[] byteArr2 = new byte[aShort2];
-        sizeB += 2;
+        short lengthB = CarbonUnsafe.getUnsafe().getShort(baseObjectR,
+            rowB + dictSizeInMemory + sizeInNonDictPartB);
+        byte[] byteArr2 = new byte[lengthB];
+        sizeInNonDictPartB += 2;
         CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObjectR, rowB + sizeB, byteArr2, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                aShort2);
-        sizeB += aShort2;
+            .copyMemory(baseObjectR, rowB + dictSizeInMemory + sizeInNonDictPartB,
+                byteArr2, CarbonUnsafe.BYTE_ARRAY_OFFSET, lengthB);
+        sizeInNonDictPartB += lengthB;
 
         int difference = UnsafeComparer.INSTANCE.compareTo(byteArr1, byteArr2);
         if (difference != 0) {
           return difference;
         }
       } else {
-        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObjectL, rowA + sizeA);
-        sizeA += 4;
-        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObjectR, rowB + sizeB);
-        sizeB += 4;
+        int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObjectL, rowA + sizeInDictPartA);
+        sizeInDictPartA += 4;
+        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObjectR, rowB + sizeInDictPartB);
+        sizeInDictPartB += 4;
         diff = dimFieldA - dimFieldB;
         if (diff != 0) {
           return diff;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
deleted file mode 100644
index 483dcb2..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.loading.sort.unsafe.comparator;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.memory.CarbonUnsafe;
-import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
-
-public class UnsafeRowComparatorForNormalDIms implements Comparator<UnsafeCarbonRow> {
-
-  private Object baseObject;
-
-  private int numberOfSortColumns;
-
-  public UnsafeRowComparatorForNormalDIms(UnsafeCarbonRowPage rowPage) {
-    this.baseObject = rowPage.getDataBlock().getBaseObject();
-    this.numberOfSortColumns = rowPage.getNoDictionarySortColumnMapping().length;
-  }
-
-  /**
-   * Below method will be used to compare two mdkey
-   */
-  public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
-    int diff = 0;
-    long rowA = rowL.address;
-    long rowB = rowR.address;
-    int sizeA = 0;
-    int sizeB = 0;
-    for (int i = 0; i < numberOfSortColumns; i++) {
-      int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
-      sizeA += 4;
-      int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
-      sizeB += 4;
-      diff = dimFieldA - dimFieldB;
-      if (diff != 0) {
-        return diff;
-      }
-    }
-
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
new file mode 100644
index 0000000..e9cfb1c
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparatorForNormalDims.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort.unsafe.comparator;
+
+import java.util.Comparator;
+
+import org.apache.carbondata.core.memory.CarbonUnsafe;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
+
+public class UnsafeRowComparatorForNormalDims implements Comparator<UnsafeCarbonRow> {
+
+  private Object baseObject;
+
+  private int numberOfSortColumns;
+
+  public UnsafeRowComparatorForNormalDims(UnsafeCarbonRowPage rowPage) {
+    this.baseObject = rowPage.getDataBlock().getBaseObject();
+    this.numberOfSortColumns = rowPage.getTableFieldStat().getIsSortColNoDictFlags().length;
+  }
+
+  /**
+   * Below method will be used to compare two mdkey
+   */
+  public int compare(UnsafeCarbonRow rowL, UnsafeCarbonRow rowR) {
+    int diff = 0;
+    long rowA = rowL.address;
+    long rowB = rowR.address;
+    int sizeA = 0;
+    int sizeB = 0;
+    for (int i = 0; i < numberOfSortColumns; i++) {
+      int dimFieldA = CarbonUnsafe.getUnsafe().getInt(baseObject, rowA + sizeA);
+      sizeA += 4;
+      int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObject, rowB + sizeB);
+      sizeB += 4;
+      diff = dimFieldA - dimFieldB;
+      if (diff != 0) {
+        return diff;
+      }
+    }
+
+    return diff;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
index 686e855..d790c41 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.java
@@ -17,6 +17,7 @@
 
 package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 /**
@@ -28,7 +29,7 @@ public interface SortTempChunkHolder extends Comparable<SortTempChunkHolder> {
 
   void readRow()  throws CarbonSortKeyAndGroupByException;
 
-  Object[] getRow();
+  IntermediateSortTempRow getRow();
 
   int numberOfRows();
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
index 6b0cfa6..a776db1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
@@ -19,9 +19,10 @@ package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeInMemoryIntermediateDataMerger;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 
 public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
 
@@ -38,21 +39,18 @@ public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
 
   private UnsafeCarbonRowPage[] rowPages;
 
-  private NewRowComparator comparator;
+  private IntermediateSortTempRowComparator comparator;
 
-  private Object[] currentRow;
-
-  private int columnSize;
+  private IntermediateSortTempRow currentRow;
 
   public UnsafeFinalMergePageHolder(UnsafeInMemoryIntermediateDataMerger merger,
-      boolean[] noDictSortColumnMapping, int columnSize) {
+      boolean[] noDictSortColumnMapping) {
     this.actualSize = merger.getEntryCount();
     this.mergedAddresses = merger.getMergedAddresses();
     this.rowPageIndexes = merger.getRowPageIndexes();
     this.rowPages = merger.getUnsafeCarbonRowPages();
     LOGGER.audit("Processing unsafe inmemory rows page with size : " + actualSize);
-    this.comparator = new NewRowComparator(noDictSortColumnMapping);
-    this.columnSize = columnSize;
+    this.comparator = new IntermediateSortTempRowComparator(noDictSortColumnMapping);
   }
 
   public boolean hasNext() {
@@ -63,12 +61,11 @@ public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
   }
 
   public void readRow() {
-    currentRow = new Object[columnSize];
-    rowPages[rowPageIndexes[counter]].getRow(mergedAddresses[counter], currentRow);
+    currentRow = rowPages[rowPageIndexes[counter]].getRow(mergedAddresses[counter]);
     counter++;
   }
 
-  public Object[] getRow() {
+  public IntermediateSortTempRow getRow() {
     return currentRow;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/2d779368/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
index 6f05088..cbcbbae 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
@@ -19,8 +19,9 @@ package org.apache.carbondata.processing.loading.sort.unsafe.holder;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.processing.loading.row.IntermediateSortTempRow;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.IntermediateSortTempRowComparator;
 
 public class UnsafeInmemoryHolder implements SortTempChunkHolder {
 
@@ -33,21 +34,18 @@ public class UnsafeInmemoryHolder implements SortTempChunkHolder {
 
   private UnsafeCarbonRowPage rowPage;
 
-  private Object[] currentRow;
+  private IntermediateSortTempRow currentRow;
 
   private long address;
 
-  private NewRowComparator comparator;
+  private IntermediateSortTempRowComparator comparator;
 
-  private int columnSize;
-
-  public UnsafeInmemoryHolder(UnsafeCarbonRowPage rowPage, int columnSize,
-      int numberOfSortColumns) {
+  public UnsafeInmemoryHolder(UnsafeCarbonRowPage rowPage) {
     this.actualSize = rowPage.getBuffer().getActualSize();
     this.rowPage = rowPage;
     LOGGER.audit("Processing unsafe inmemory rows page with size : " + actualSize);
-    this.comparator = new NewRowComparator(rowPage.getNoDictionarySortColumnMapping());
-    this.columnSize = columnSize;
+    this.comparator = new IntermediateSortTempRowComparator(
+        rowPage.getTableFieldStat().getIsSortColNoDictFlags());
   }
 
   public boolean hasNext() {
@@ -58,13 +56,12 @@ public class UnsafeInmemoryHolder implements SortTempChunkHolder {
   }
 
   public void readRow() {
-    currentRow = new Object[columnSize];
     address = rowPage.getBuffer().get(counter);
-    rowPage.getRow(address + rowPage.getDataBlock().getBaseOffset(), currentRow);
+    currentRow = rowPage.getRow(address + rowPage.getDataBlock().getBaseOffset());
     counter++;
   }
 
-  public Object[] getRow() {
+  public IntermediateSortTempRow getRow() {
     return currentRow;
   }
 


[11/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
index 447ab46..547ecaa 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
 import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
 import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
 import org.apache.carbondata.core.datastore.page.ColumnPage;
@@ -35,12 +35,11 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.scan.expression.Expression;
-import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.FilterUtil;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.apache.carbondata.core.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -73,7 +72,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       comparator = Comparator.getComparatorByDataTypeForMeasure(measure.getDataType());
     }
     ifDefaultValueMatchesFilter();
-    if (isDimensionPresentInCurrentBlock[0] == true) {
+    if (isDimensionPresentInCurrentBlock[0]) {
       isNaturalSorted = dimColEvaluatorInfoList.get(0).getDimension().isUseInvertedIndex()
           && dimColEvaluatorInfoList.get(0).getDimension().isSortColumn();
     }
@@ -120,11 +119,11 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     boolean isScanRequired = false;
     if (isMeasurePresentInCurrentBlock[0] || isDimensionPresentInCurrentBlock[0]) {
       if (isMeasurePresentInCurrentBlock[0]) {
-        minValue = blockMinValue[measureBlocksIndex[0] + lastDimensionColOrdinal];
+        minValue = blockMinValue[measureChunkIndex[0] + lastDimensionColOrdinal];
         isScanRequired =
             isScanRequired(minValue, msrFilterRangeValues, msrColEvalutorInfoList.get(0).getType());
       } else {
-        minValue = blockMinValue[dimensionBlocksIndex[0]];
+        minValue = blockMinValue[dimensionChunkIndex[0]];
         isScanRequired = isScanRequired(minValue, filterRangeValues);
       }
     } else {
@@ -170,67 +169,69 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
   }
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws IOException {
     // select all rows if dimension does not exists in the current block
     if (!isDimensionPresentInCurrentBlock[0] && !isMeasurePresentInCurrentBlock[0]) {
-      int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+      int numberOfRows = rawBlockletColumnChunks.getDataBlock().numRows();
       return FilterUtil
-          .createBitSetGroupWithDefaultValue(blockChunkHolder.getDataBlock().numberOfPages(),
+          .createBitSetGroupWithDefaultValue(rawBlockletColumnChunks.getDataBlock().numberOfPages(),
               numberOfRows, true);
     }
     if (isDimensionPresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getDimensionOrdinalToBlockMapping().get(dimensionBlocksIndex[0]);
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex =
+          segmentProperties.getDimensionOrdinalToChunkMapping().get(dimensionChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
       DimensionRawColumnChunk rawColumnChunk =
-          blockChunkHolder.getDimensionRawDataChunk()[blockIndex];
+          rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMinValues() != null) {
           if (isScanRequired(rawColumnChunk.getMinValues()[i], this.filterRangeValues)) {
-            BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+            BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
                 rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
-          BitSet bitSet = getFilteredIndexes(rawColumnChunk.convertToDimColDataChunk(i),
+          BitSet bitSet = getFilteredIndexes(rawColumnChunk.decodeColumnPage(i),
               rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
-    } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex =
-          segmentProperties.getMeasuresOrdinalToBlockMapping().get(measureBlocksIndex[0]);
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+    } else {
+      int chunkIndex =
+          segmentProperties.getMeasuresOrdinalToChunkMapping().get(measureChunkIndex[0]);
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
-      MeasureRawColumnChunk rawColumnChunk = blockChunkHolder.getMeasureRawDataChunk()[blockIndex];
+      MeasureRawColumnChunk rawColumnChunk =
+          rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex];
       BitSetGroup bitSetGroup = new BitSetGroup(rawColumnChunk.getPagesCount());
       for (int i = 0; i < rawColumnChunk.getPagesCount(); i++) {
         if (rawColumnChunk.getMinValues() != null) {
           if (isScanRequired(rawColumnChunk.getMinValues()[i], this.msrFilterRangeValues,
               msrColEvalutorInfoList.get(0).getType())) {
             BitSet bitSet =
-                getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+                getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                     rawColumnChunk.getRowCount()[i]);
             bitSetGroup.setBitSet(bitSet, i);
           }
         } else {
           BitSet bitSet =
-              getFilteredIndexesForMeasures(rawColumnChunk.convertToColumnPage(i),
+              getFilteredIndexesForMeasures(rawColumnChunk.decodeColumnPage(i),
                   rawColumnChunk.getRowCount()[i]);
           bitSetGroup.setBitSet(bitSet, i);
         }
       }
       return bitSetGroup;
     }
-    return null;
   }
 
   @Override
@@ -280,7 +281,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     return bitSet;
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet getFilteredIndexes(DimensionColumnPage dimensionColumnPage,
       int numerOfRows) {
     byte[] defaultValue = null;
     if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
@@ -289,7 +290,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
               dimColEvaluatorInfoList.get(0).getDimension().getDataType());
       int key = directDictionaryGenerator.generateDirectSurrogateKey(null);
       CarbonDimension currentBlockDimension =
-          segmentProperties.getDimensions().get(dimensionBlocksIndex[0]);
+          segmentProperties.getDimensions().get(dimensionChunkIndex[0]);
       if (currentBlockDimension.isSortColumn()) {
         defaultValue = FilterUtil.getMaskKey(key, currentBlockDimension,
             this.segmentProperties.getSortColumnsGenerator());
@@ -300,14 +301,14 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       defaultValue = CarbonCommonConstants.EMPTY_BYTE_ARRAY;
     }
     BitSet bitSet = null;
-    if (dimensionColumnDataChunk.isExplicitSorted()) {
-      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnDataChunk, numerOfRows,
+    if (dimensionColumnPage.isExplicitSorted()) {
+      bitSet = setFilterdIndexToBitSetWithColumnIndex(dimensionColumnPage, numerOfRows,
           defaultValue);
     } else {
-      bitSet = setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue);
+      bitSet = setFilterdIndexToBitSet(dimensionColumnPage, numerOfRows, defaultValue);
     }
-    if (dimensionColumnDataChunk.isNoDicitionaryColumn()) {
-      FilterUtil.removeNullValues(dimensionColumnDataChunk, bitSet,
+    if (dimensionColumnPage.isNoDicitionaryColumn()) {
+      FilterUtil.removeNullValues(dimensionColumnPage, bitSet,
           CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY);
     }
     return bitSet;
@@ -319,12 +320,12 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
    * column is not supported by default so column index mapping  will be present for
    * accesing the members from the block.
    *
-   * @param dimensionColumnDataChunk
+   * @param dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      DimensionColumnDataChunk dimensionColumnDataChunk, int numerOfRows,
+      DimensionColumnPage dimensionColumnPage, int numerOfRows,
       byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     int start = 0;
@@ -336,7 +337,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     //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,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               defaultValue, true);
       if (start < 0) {
         skip = -(start + 1);
@@ -354,7 +355,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
 
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+          .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex, numerOfRows - 1,
               filterValues[i], false);
       if (start >= 0) {
         // Logic will handle the case where the range filter member is not present in block
@@ -362,7 +363,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
         // 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]);
+            CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[i]);
       }
       if (start < 0) {
         start = -(start + 1);
@@ -373,14 +374,14 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
         // will be pointing to the next consecutive position. So compare it again and point to the
         // previous value returned from getFirstIndexUsingBinarySearch.
         if (ByteUtil.compare(filterValues[i],
-            dimensionColumnDataChunk.getChunkData(dimensionColumnDataChunk.getInvertedIndex(start)))
+            dimensionColumnPage.getChunkData(dimensionColumnPage.getInvertedIndex(start)))
             < 0) {
           start = start - 1;
         }
       }
       last = start;
       for (int j = start; j >= skip; j--) {
-        bitSet.set(dimensionColumnDataChunk.getInvertedIndex(j));
+        bitSet.set(dimensionColumnPage.getInvertedIndex(j));
         last--;
       }
       startIndex = last;
@@ -397,11 +398,11 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
    * 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 dimensionColumnPage
    * @param numerOfRows
    * @return BitSet.
    */
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+  private BitSet setFilterdIndexToBitSet(DimensionColumnPage dimensionColumnPage,
       int numerOfRows, byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     byte[][] filterValues = this.filterRangeValues;
@@ -414,7 +415,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       //find the number of default values to skip the null value in case of direct dictionary
       if (null != defaultValue) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, defaultValue, true);
         if (start < 0) {
           skip = -(start + 1);
@@ -431,11 +432,11 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
       }
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil
-            .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex,
+            .getFirstIndexUsingBinarySearch(dimensionColumnPage, startIndex,
                 numerOfRows - 1, filterValues[k], false);
         if (start >= 0) {
           start =
-              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[k]);
+              CarbonUtil.nextLesserValueToTarget(start, dimensionColumnPage, filterValues[k]);
         }
         if (start < 0) {
           start = -(start + 1);
@@ -446,7 +447,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
           // When negative value of start is returned from getFirstIndexUsingBinarySearch the Start
           // will be pointing to the next consecutive position. So compare it again and point to the
           // previous value returned from getFirstIndexUsingBinarySearch.
-          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) < 0) {
+          if (ByteUtil.compare(filterValues[k], dimensionColumnPage.getChunkData(start)) < 0) {
             start = start - 1;
           }
         }
@@ -463,7 +464,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     } else {
       for (int k = 0; k < filterValues.length; k++) {
         for (int i = 0; i < numerOfRows; i++) {
-          if (ByteUtil.compare(dimensionColumnDataChunk.getChunkData(i), filterValues[k]) < 0) {
+          if (ByteUtil.compare(dimensionColumnPage.getChunkData(i), filterValues[k]) < 0) {
             bitSet.set(i);
           }
         }
@@ -472,21 +473,24 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     return bitSet;
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
     if (isDimensionPresentInCurrentBlock[0]) {
       if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        super.readBlocks(blockChunkHolder);
+        super.readColumnChunks(rawBlockletColumnChunks);
       }
-      int blockIndex = dimensionBlocksIndex[0];
-      if (null == blockChunkHolder.getDimensionRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getDimensionRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = dimensionChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getDimensionRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readDimensionChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     } else if (isMeasurePresentInCurrentBlock[0]) {
-      int blockIndex = measureBlocksIndex[0];
-      if (null == blockChunkHolder.getMeasureRawDataChunk()[blockIndex]) {
-        blockChunkHolder.getMeasureRawDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-            .getMeasureChunk(blockChunkHolder.getFileReader(), blockIndex);
+      int chunkIndex = measureChunkIndex[0];
+      if (null == rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex]) {
+        rawBlockletColumnChunks.getMeasureRawColumnChunks()[chunkIndex] =
+            rawBlockletColumnChunks.getDataBlock().readMeasureChunk(
+                rawBlockletColumnChunks.getFileReader(), chunkIndex);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
index 4b3738a..2cbb234 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/TrueFilterExecutor.java
@@ -22,7 +22,7 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 public class TrueFilterExecutor implements FilterExecuter {
@@ -33,19 +33,20 @@ public class TrueFilterExecutor implements FilterExecuter {
    * @return
    * @throws FilterUnsupportedException
    */
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-    int numberOfPages = blockChunkHolder.getDataBlock().numberOfPages();
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
+    int numberOfPages = rawBlockletColumnChunks.getDataBlock().numberOfPages();
     BitSetGroup group = new BitSetGroup(numberOfPages);
     for (int i = 0; i < numberOfPages; i++) {
       BitSet set = new BitSet();
-      set.flip(0, blockChunkHolder.getDataBlock().getPageRowCount(i));
+      set.flip(0, rawBlockletColumnChunks.getDataBlock().getPageRowCount(i));
       group.setBitSet(set, i);
     }
     return group;
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax) {
     return true;
   }
 
@@ -66,9 +67,9 @@ public class TrueFilterExecutor implements FilterExecuter {
   /**
    * It just reads necessary block for filter executor, it does not uncompress the data.
    *
-   * @param blockChunkHolder
+   * @param rawBlockletColumnChunks
    */
-  public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) {
     // do nothing
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
index 516447f..bc3b39c 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/executer/ValueBasedFilterExecuterImpl.java
@@ -21,27 +21,29 @@ import java.util.BitSet;
 
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
 import org.apache.carbondata.core.scan.filter.intf.RowIntf;
-import org.apache.carbondata.core.scan.processor.BlocksChunkHolder;
+import org.apache.carbondata.core.scan.processor.RawBlockletColumnChunks;
 import org.apache.carbondata.core.util.BitSetGroup;
 
 public class ValueBasedFilterExecuterImpl implements FilterExecuter {
 
   @Override
-  public BitSetGroup applyFilter(BlocksChunkHolder blockChunkHolder, boolean useBitsetPipeLine)
-      throws FilterUnsupportedException, IOException {
-
+  public BitSetGroup applyFilter(RawBlockletColumnChunks rawBlockletColumnChunks,
+      boolean useBitsetPipeLine) throws FilterUnsupportedException, IOException {
     return new BitSetGroup(0);
   }
 
-  @Override public boolean applyFilter(RowIntf value, int dimOrdinalMax)
+  @Override
+  public boolean applyFilter(RowIntf value, int dimOrdinalMax)
       throws FilterUnsupportedException, IOException {
     throw new FilterUnsupportedException("Unsupported ValueBasedFilterExecuterImpl on row");
   }
 
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+  @Override
+  public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
     return new BitSet(1);
   }
 
-  @Override public void readBlocks(BlocksChunkHolder blockChunkHolder) throws IOException {
+  @Override
+  public void readColumnChunks(RawBlockletColumnChunks rawBlockletColumnChunks) throws IOException {
   }
 }

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java
index e5cb0ad..bbd33ec 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/optimizer/RangeFilterOptmizer.java
@@ -19,14 +19,13 @@ package org.apache.carbondata.core.scan.filter.optimizer;
 
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.RangeExpressionEvaluator;
-import org.apache.carbondata.core.scan.filter.intf.FilterOptimizer;
 import org.apache.carbondata.core.scan.filter.intf.FilterOptimizerImpl;
 
 public class RangeFilterOptmizer extends FilterOptimizerImpl {
 
   RangeExpressionEvaluator rangeExpEvaluator;
 
-  public RangeFilterOptmizer(FilterOptimizer filterOptimizer, Expression filterExpression) {
+  public RangeFilterOptmizer(Expression filterExpression) {
     super(filterExpression);
     this.rangeExpEvaluator = new RangeExpressionEvaluator(filterExpression);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
index bb66d18..996965d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/ConditionalFilterResolverImpl.java
@@ -46,17 +46,13 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
   protected boolean isIncludeFilter;
   private DimColumnResolvedFilterInfo dimColResolvedFilterInfo;
   private MeasureColumnResolvedFilterInfo msrColResolvedFilterInfo;
-  private AbsoluteTableIdentifier tableIdentifier;
-  private boolean isMeasure;
 
   public ConditionalFilterResolverImpl(Expression exp, boolean isExpressionResolve,
-      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier, boolean isMeasure) {
+      boolean isIncludeFilter, boolean isMeasure) {
     this.exp = exp;
     this.isExpressionResolve = isExpressionResolve;
     this.isIncludeFilter = isIncludeFilter;
-    this.tableIdentifier = tableIdentifier;
-    this.isMeasure = isMeasure;
-    if (isMeasure == false) {
+    if (!isMeasure) {
       this.dimColResolvedFilterInfo = new DimColumnResolvedFilterInfo();
     } else {
       this.msrColResolvedFilterInfo = new MeasureColumnResolvedFilterInfo();
@@ -230,10 +226,6 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
     return msrColResolvedFilterInfo;
   }
 
-  public AbsoluteTableIdentifier getTableIdentifier() {
-    return tableIdentifier;
-  }
-
   /**
    * method will calculates the start key based on the filter surrogates
    */
@@ -245,12 +237,6 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
       FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
           setOfStartKeyByteArray);
     }
-// else {
-//      FilterUtil.getStartKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
-//          segmentProperties, startKey, startKeyList);
-//      FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
-//          setOfStartKeyByteArray);
-//    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
index 33bafb1..c846c02 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelFilterResolverImpl.java
@@ -40,7 +40,7 @@ public class RowLevelFilterResolverImpl extends ConditionalFilterResolverImpl {
 
   public RowLevelFilterResolverImpl(Expression exp, boolean isExpressionResolve,
       boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
-    super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false);
+    super(exp, isExpressionResolve, isIncludeFilter, false);
     dimColEvaluatorInfoList =
         new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
index 0e7666c..e85e9d1 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
@@ -61,7 +61,7 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
 
   public RowLevelRangeFilterResolverImpl(Expression exp, boolean isExpressionResolve,
       boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
-    super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false);
+    super(exp, isExpressionResolve, isIncludeFilter, false);
     dimColEvaluatorInfoList =
         new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
index f8586ed..4165424 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
@@ -77,10 +77,6 @@ public class MeasureColumnResolvedFilterInfo extends ColumnResolvedFilterInfo
     }
   }
 
-  public Map<CarbonMeasure, List<ColumnFilterInfo>> getMeasureResolvedFilterInstance() {
-    return measureResolvedFilter;
-  }
-
   public ColumnFilterInfo getFilterValues() {
     return resolvedFilterValueObj;
   }
@@ -109,10 +105,6 @@ public class MeasureColumnResolvedFilterInfo extends ColumnResolvedFilterInfo
     this.type = dataType;
   }
 
-  public boolean isMeasureExistsInCurrentSlice() {
-    return true;
-  }
-
   public CarbonColumn getCarbonColumn() {
     return carbonColumn;
   }
@@ -125,10 +117,6 @@ public class MeasureColumnResolvedFilterInfo extends ColumnResolvedFilterInfo
     return carbonMeasure;
   }
 
-  public boolean isMeasureExistsInCurrentSilce() {
-    return isMeasureExistsInCurrentSilce;
-  }
-
   public void setMeasureExistsInCurrentSilce(boolean measureExistsInCurrentSilce) {
     isMeasureExistsInCurrentSilce = measureExistsInCurrentSilce;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
index d403ef4..104f69e 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/TrueConditionalResolverImpl.java
@@ -28,9 +28,9 @@ import org.apache.carbondata.core.scan.filter.resolver.ConditionalFilterResolver
 public class TrueConditionalResolverImpl extends ConditionalFilterResolverImpl {
 
   public TrueConditionalResolverImpl(Expression exp, boolean isExpressionResolve,
-      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
+      boolean isIncludeFilter) {
 
-    super(exp, isExpressionResolve, isIncludeFilter, tableIdentifier, false);
+    super(exp, isExpressionResolve, isIncludeFilter, false);
   }
 
   @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier,
@@ -47,16 +47,4 @@ public class TrueConditionalResolverImpl extends ConditionalFilterResolverImpl {
     return FilterExecuterType.TRUE;
   }
 
-  /**
-   * Method will the read filter expression corresponding to the resolver.
-   * This method is required in row level executer inorder to evaluate the filter
-   * expression against spark, as mentioned above row level is a special type
-   * filter resolver.
-   *
-   * @return Expression
-   */
-  public Expression getFilterExpresion() {
-    return exp;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java
index c90816a..b1e5a7d 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/filter/resolver/resolverinfo/visitor/RangeDictionaryColumnVisitor.java
@@ -19,7 +19,6 @@ package org.apache.carbondata.core.scan.filter.resolver.resolverinfo.visitor;
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.List;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.scan.expression.exception.FilterUnsupportedException;
@@ -47,7 +46,6 @@ public class RangeDictionaryColumnVisitor extends DictionaryColumnVisitor
     if (visitableObj instanceof DimColumnResolvedFilterInfo) {
       DimColumnResolvedFilterInfo resolveDimension = (DimColumnResolvedFilterInfo) visitableObj;
       ColumnFilterInfo resolvedFilterObject = null;
-      List<String> evaluateResultListFinal;
       resolvedFilterObject = FilterUtil
           .getFilterListForAllValues(metadata.getTableIdentifier(), metadata.getExpression(),
               metadata.getColumnExpression(), metadata.isIncludeFilter(),

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java b/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java
deleted file mode 100644
index 5a5b042..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/CarbonQueryPlan.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.model;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.scan.expression.Expression;
-
-/**
- * This class contains all the logical information about the query like dimensions,measures,
- * sort order, topN etc..
- */
-public class CarbonQueryPlan implements Serializable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = -9036044826928017164L;
-
-  /**
-   * Database name
-   */
-  private String databaseName;
-
-  /**
-   * Table name
-   */
-  private String tableName;
-
-  /**
-   * List of dimensions.
-   * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name
-   * and department_name are dimensions
-   * If there is no dimensions asked in query then it would be remained as empty.
-   */
-  private List<QueryDimension> dimensions =
-      new ArrayList<QueryDimension>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  /**
-   * List of measures.
-   * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary)
-   * would be measure.
-   * If there is no dimensions asked in query then it would be remained as empty.
-   */
-  private List<QueryMeasure> measures =
-      new ArrayList<QueryMeasure>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  /**
-   * expression
-   */
-  private Expression expression;
-
-  /**
-   * queryId
-   */
-  private String queryId;
-
-  /**
-   * If it is raw detail query, no need to aggregate in backend. And it returns with dictionary data
-   * with out decoding.
-   */
-  private boolean rawDetailQuery;
-
-  /**
-   * Constructor created with database name and table name.
-   *
-   * @param databaseName
-   * @param tableName
-   */
-  public CarbonQueryPlan(String databaseName, String tableName) {
-    this.tableName = tableName;
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * @return the dimensions
-   */
-  public List<QueryDimension> getDimensions() {
-    return dimensions;
-  }
-
-  public void addDimension(QueryDimension dimension) {
-    this.dimensions.add(dimension);
-  }
-
-  /**
-   * @return the measures
-   */
-  public List<QueryMeasure> getMeasures() {
-    return measures;
-  }
-
-  public void addMeasure(QueryMeasure measure) {
-    this.measures.add(measure);
-  }
-
-  public Expression getFilterExpression() {
-    return expression;
-  }
-
-  public void setFilterExpression(Expression expression) {
-    this.expression = expression;
-  }
-
-  /**
-   * @return the databaseName
-   */
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  /**
-   * @return the tableName
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  public String getQueryId() {
-    return queryId;
-  }
-
-  public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  public boolean isRawDetailQuery() {
-    return rawDetailQuery;
-  }
-
-  public void setRawDetailQuery(boolean rawDetailQuery) {
-    this.rawDetailQuery = rawDetailQuery;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java
new file mode 100644
index 0000000..0768221
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionColumn.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.model;
+
+/**
+ * Contains information for a column for projection
+ */
+public class ProjectionColumn {
+
+  /**
+   * name of the column
+   */
+  protected String columnName;
+
+  /**
+   * query order in which result of the query will be send
+   */
+  private int projectionOrdinal;
+
+  ProjectionColumn(String columnName) {
+    this.columnName = columnName;
+  }
+
+  /**
+   * @return the columnName
+   */
+  public String getColumnName() {
+    return columnName;
+  }
+
+  /**
+   * @return the projectionOrdinal
+   */
+  public int getOrdinal() {
+    return projectionOrdinal;
+  }
+
+  /**
+   * @param projectionOrdinal the projectionOrdinal to set
+   */
+  public void setOrdinal(int projectionOrdinal) {
+    this.projectionOrdinal = projectionOrdinal;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java
new file mode 100644
index 0000000..806de85
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/ProjectionDimension.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.scan.model;
+
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+
+/**
+ * dimension column which holds the information about the column
+ * this is done to avoid heavy object serialization
+ */
+public class ProjectionDimension extends ProjectionColumn {
+
+  /**
+   * actual dimension column
+   */
+  private CarbonDimension dimension;
+
+  public ProjectionDimension(CarbonDimension dimension) {
+    super(dimension.getColName());
+    this.dimension = dimension;
+  }
+
+  /**
+   * @return the dimension
+   */
+  public CarbonDimension getDimension() {
+    return dimension;
+  }
+
+}

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java
deleted file mode 100644
index 1336b87..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryColumn.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.model;
-
-import java.io.Serializable;
-
-/**
- * query column  which will have information about column
- */
-public class QueryColumn implements Serializable {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -4222306600480181084L;
-
-  /**
-   * name of the column
-   */
-  protected String columnName;
-
-  /**
-   * query order in which result of the query will be send
-   */
-  private int queryOrder;
-
-  public QueryColumn(String columnName) {
-    this.columnName = columnName;
-  }
-
-  /**
-   * @return the columnName
-   */
-  public String getColumnName() {
-    return columnName;
-  }
-
-  /**
-   * @return the queryOrder
-   */
-  public int getQueryOrder() {
-    return queryOrder;
-  }
-
-  /**
-   * @param queryOrder the queryOrder to set
-   */
-  public void setQueryOrder(int queryOrder) {
-    this.queryOrder = queryOrder;
-  }
-
-}
\ No newline at end of file

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

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
index 655ed5e..4e3665f 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.core.scan.model;
 
-import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -46,25 +45,17 @@ import org.apache.carbondata.core.util.DataTypeConverter;
  * about the query, This will be sent from driver to executor '
  * This will be refereed to executing the query.
  */
-public class QueryModel implements Serializable {
+public class QueryModel {
 
   /**
-   * serialization version
-   */
-  private static final long serialVersionUID = -4674677234007089052L;
-  /**
    * this will hold the information about the dictionary dimension
    * which to
    */
-  public transient Map<String, Dictionary> columnToDictionaryMapping;
-  /**
-   * list of dimension selected for in query
-   */
-  private List<QueryDimension> queryDimension;
+  private transient Map<String, Dictionary> columnToDictionaryMapping;
   /**
-   * list of measure selected in query
+   * list of projection columns in query
    */
-  private List<QueryMeasure> queryMeasures;
+  private QueryProjection projection;
   /**
    * query id
    */
@@ -79,10 +70,6 @@ public class QueryModel implements Serializable {
    */
   private List<TableBlockInfo> tableBlockInfos;
   /**
-   * absolute table identifier
-   */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-  /**
    * To handle most of the computation in query engines like spark and hive, carbon should give
    * raw detailed records to it.
    */
@@ -117,43 +104,15 @@ public class QueryModel implements Serializable {
    */
   private boolean readPageByPage;
 
-  public QueryModel() {
+  private QueryModel(CarbonTable carbonTable) {
     tableBlockInfos = new ArrayList<TableBlockInfo>();
-    queryDimension = new ArrayList<QueryDimension>();
-    queryMeasures = new ArrayList<QueryMeasure>();
     invalidSegmentIds = new ArrayList<>();
+    this.table = carbonTable;
+    this.queryId = String.valueOf(System.nanoTime());
   }
 
-  public static QueryModel createModel(AbsoluteTableIdentifier absoluteTableIdentifier,
-      CarbonQueryPlan queryPlan, CarbonTable carbonTable, DataTypeConverter converter) {
-    QueryModel queryModel = new QueryModel();
-    queryModel.setAbsoluteTableIdentifier(absoluteTableIdentifier);
-
-    fillQueryModel(queryPlan, carbonTable, queryModel);
-
-    queryModel.setForcedDetailRawQuery(queryPlan.isRawDetailQuery());
-    queryModel.setQueryId(queryPlan.getQueryId());
-    queryModel.setConverter(converter);
-    return queryModel;
-  }
-
-  private static void fillQueryModel(CarbonQueryPlan queryPlan, CarbonTable carbonTable,
-      QueryModel queryModel) {
-    queryModel.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
-    queryModel.setQueryDimension(queryPlan.getDimensions());
-    queryModel.setQueryMeasures(queryPlan.getMeasures());
-    if (null != queryPlan.getFilterExpression()) {
-      boolean[] isFilterDimensions = new boolean[carbonTable.getDimensionOrdinalMax()];
-      boolean[] isFilterMeasures =
-          new boolean[carbonTable.getNumberOfMeasures(carbonTable.getTableName())];
-      processFilterExpression(carbonTable, queryPlan.getFilterExpression(), isFilterDimensions,
-          isFilterMeasures);
-      queryModel.setIsFilterDimensions(isFilterDimensions);
-      queryModel.setIsFilterMeasures(isFilterMeasures);
-    }
-    //TODO need to remove this code, and executor will load the table
-    // from file metadata
-    queryModel.setTable(carbonTable);
+  public static QueryModel newInstance(CarbonTable carbonTable) {
+    return new QueryModel(carbonTable);
   }
 
   public static void processFilterExpression(CarbonTable carbonTable, Expression filterExpression,
@@ -239,42 +198,26 @@ public class QueryModel implements Serializable {
    */
   public CarbonColumn[] getProjectionColumns() {
     CarbonColumn[] carbonColumns =
-        new CarbonColumn[getQueryDimension().size() + getQueryMeasures().size()];
-    for (QueryDimension dimension : getQueryDimension()) {
-      carbonColumns[dimension.getQueryOrder()] = dimension.getDimension();
+        new CarbonColumn[getProjectionDimensions().size() + getProjectionMeasures().size()];
+    for (ProjectionDimension dimension : getProjectionDimensions()) {
+      carbonColumns[dimension.getOrdinal()] = dimension.getDimension();
     }
-    for (QueryMeasure msr : getQueryMeasures()) {
-      carbonColumns[msr.getQueryOrder()] = msr.getMeasure();
+    for (ProjectionMeasure msr : getProjectionMeasures()) {
+      carbonColumns[msr.getOrdinal()] = msr.getMeasure();
     }
     return carbonColumns;
   }
 
-  /**
-   * @return the queryDimension
-   */
-  public List<QueryDimension> getQueryDimension() {
-    return queryDimension;
+  public void setProjection(QueryProjection projection) {
+    this.projection = projection;
   }
 
-  /**
-   * @param queryDimension the queryDimension to set
-   */
-  public void setQueryDimension(List<QueryDimension> queryDimension) {
-    this.queryDimension = queryDimension;
+  public List<ProjectionDimension> getProjectionDimensions() {
+    return projection.getDimensions();
   }
 
-  /**
-   * @return the queryMeasures
-   */
-  public List<QueryMeasure> getQueryMeasures() {
-    return queryMeasures;
-  }
-
-  /**
-   * @param queryMeasures the queryMeasures to set
-   */
-  public void setQueryMeasures(List<QueryMeasure> queryMeasures) {
-    this.queryMeasures = queryMeasures;
+  public List<ProjectionMeasure> getProjectionMeasures() {
+    return projection.getMeasures();
   }
 
   /**
@@ -285,13 +228,6 @@ public class QueryModel implements Serializable {
   }
 
   /**
-   * @param queryId the queryId to set
-   */
-  public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  /**
    * @return the tableBlockInfos
    */
   public List<TableBlockInfo> getTableBlockInfos() {
@@ -320,14 +256,7 @@ public class QueryModel implements Serializable {
    * @return the absoluteTableIdentifier
    */
   public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  /**
-   * @param absoluteTableIdentifier the absoluteTableIdentifier to set
-   */
-  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
+    return table.getAbsoluteTableIdentifier();
   }
 
   /**
@@ -337,13 +266,6 @@ public class QueryModel implements Serializable {
     return table;
   }
 
-  /**
-   * @param table the table to set
-   */
-  public void setTable(CarbonTable table) {
-    this.table = table;
-  }
-
   public boolean isForcedDetailRawQuery() {
     return forcedDetailRawQuery;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java
new file mode 100644
index 0000000..d4496e0
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryProjection.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.scan.model;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+
+/**
+ * Contains projection columns in the query
+ */
+public class QueryProjection {
+
+  /**
+   * List of dimensions.
+   * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name
+   * and department_name are dimensions
+   * If there is no dimensions asked in query then it would be remained as empty.
+   */
+  private List<ProjectionDimension> dimensions =
+      new ArrayList<ProjectionDimension>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  /**
+   * List of measures.
+   * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary)
+   * would be measure.
+   * If there is no dimensions asked in query then it would be remained as empty.
+   */
+  private List<ProjectionMeasure> measures =
+      new ArrayList<ProjectionMeasure>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  /**
+   * Constructor created with database name and table name.
+   *
+   */
+  public QueryProjection() {
+  }
+
+  /**
+   * @return the dimensions
+   */
+  public List<ProjectionDimension> getDimensions() {
+    return dimensions;
+  }
+
+  public void addDimension(CarbonDimension dimension, int queryOrdinal) {
+    ProjectionDimension queryDimension = new ProjectionDimension(dimension);
+    queryDimension.setOrdinal(queryOrdinal);
+    this.dimensions.add(queryDimension);
+  }
+
+  /**
+   * @return the measures
+   */
+  public List<ProjectionMeasure> getMeasures() {
+    return measures;
+  }
+
+  public void addMeasure(CarbonMeasure measure, int queryOrdinal) {
+    ProjectionMeasure queryMeasure = new ProjectionMeasure(measure);
+    queryMeasure.setOrdinal(queryOrdinal);
+    this.measures.add(queryMeasure);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java
deleted file mode 100644
index eb5e3f9..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/processor/AbstractDataBlockIterator.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.processor;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.datastore.DataRefNode;
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.scan.collector.ResultCollectorFactory;
-import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.scan.scanner.BlockletScanner;
-import org.apache.carbondata.core.scan.scanner.impl.FilterScanner;
-import org.apache.carbondata.core.scan.scanner.impl.NonFilterScanner;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-import org.apache.carbondata.core.util.TaskMetricsMap;
-
-/**
- * This abstract class provides a skeletal implementation of the
- * Block iterator.
- */
-public abstract class AbstractDataBlockIterator extends CarbonIterator<List<Object[]>> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractDataBlockIterator.class.getName());
-
-  /**
-   * iterator which will be used to iterate over data blocks
-   */
-  protected CarbonIterator<DataRefNode> dataBlockIterator;
-
-  /**
-   * result collector which will be used to aggregate the scanned result
-   */
-  protected ScannedResultCollector scannerResultAggregator;
-
-  /**
-   * processor which will be used to process the block processing can be
-   * filter processing or non filter processing
-   */
-  protected BlockletScanner blockletScanner;
-
-  /**
-   * batch size of result
-   */
-  protected int batchSize;
-
-  protected ExecutorService executorService;
-
-  private Future<AbstractScannedResult> future;
-
-  private Future<BlocksChunkHolder> futureIo;
-
-  protected AbstractScannedResult scannedResult;
-
-  private BlockExecutionInfo blockExecutionInfo;
-
-  private FileHolder fileReader;
-
-  private AtomicBoolean nextBlock;
-
-  private AtomicBoolean nextRead;
-
-  public AbstractDataBlockIterator(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader,
-      int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) {
-    this.blockExecutionInfo = blockExecutionInfo;
-    this.fileReader = fileReader;
-    dataBlockIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
-        blockExecutionInfo.getNumberOfBlockToScan());
-    if (blockExecutionInfo.getFilterExecuterTree() != null) {
-      blockletScanner = new FilterScanner(blockExecutionInfo, queryStatisticsModel);
-    } else {
-      blockletScanner = new NonFilterScanner(blockExecutionInfo, queryStatisticsModel);
-    }
-    this.scannerResultAggregator =
-        ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo);
-    this.batchSize = batchSize;
-    this.executorService = executorService;
-    this.nextBlock = new AtomicBoolean(false);
-    this.nextRead = new AtomicBoolean(false);
-  }
-
-  public boolean hasNext() {
-    if (scannedResult != null && scannedResult.hasNext()) {
-      return true;
-    } else {
-      if (null != scannedResult) {
-        scannedResult.freeMemory();
-      }
-      return dataBlockIterator.hasNext() || nextBlock.get() || nextRead.get();
-    }
-  }
-
-  protected boolean updateScanner() {
-    try {
-      if (scannedResult != null && scannedResult.hasNext()) {
-        return true;
-      } else {
-        scannedResult = getNextScannedResult();
-        while (scannedResult != null) {
-          if (scannedResult.hasNext()) {
-            return true;
-          }
-          scannedResult = getNextScannedResult();
-        }
-        nextBlock.set(false);
-        nextRead.set(false);
-        return false;
-      }
-    } catch (Exception ex) {
-      throw new RuntimeException(ex);
-    }
-  }
-
-  private AbstractScannedResult getNextScannedResult() throws Exception {
-    AbstractScannedResult result = null;
-    if (blockExecutionInfo.isPrefetchBlocklet()) {
-      if (dataBlockIterator.hasNext() || nextBlock.get() || nextRead.get()) {
-        if (future == null) {
-          future = execute();
-        }
-        result = future.get();
-        nextBlock.set(false);
-        if (dataBlockIterator.hasNext() || nextRead.get()) {
-          nextBlock.set(true);
-          future = execute();
-        }
-      }
-    } else {
-      if (dataBlockIterator.hasNext()) {
-        BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolder();
-        if (blocksChunkHolder != null) {
-          result = blockletScanner.scanBlocklet(blocksChunkHolder);
-        }
-      }
-    }
-    return result;
-  }
-
-  private BlocksChunkHolder getBlocksChunkHolder() throws IOException {
-    BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolderInternal();
-    while (blocksChunkHolder == null && dataBlockIterator.hasNext()) {
-      blocksChunkHolder = getBlocksChunkHolderInternal();
-    }
-    return blocksChunkHolder;
-  }
-
-  private BlocksChunkHolder getBlocksChunkHolderInternal() throws IOException {
-    BlocksChunkHolder blocksChunkHolder =
-        new BlocksChunkHolder(blockExecutionInfo.getTotalNumberDimensionBlock(),
-            blockExecutionInfo.getTotalNumberOfMeasureBlock(), fileReader);
-    blocksChunkHolder.setDataBlock(dataBlockIterator.next());
-    if (blocksChunkHolder.getDataBlock().getColumnsMaxValue() == null) {
-      return blocksChunkHolder;
-    }
-    if (blockletScanner.isScanRequired(blocksChunkHolder)) {
-      return blocksChunkHolder;
-    }
-    return null;
-  }
-
-  private Future<AbstractScannedResult> execute() {
-    return executorService.submit(new Callable<AbstractScannedResult>() {
-      @Override public AbstractScannedResult call() throws Exception {
-        if (futureIo == null) {
-          futureIo = executeRead();
-        }
-        BlocksChunkHolder blocksChunkHolder = futureIo.get();
-        futureIo = null;
-        nextRead.set(false);
-        if (blocksChunkHolder != null) {
-          if (dataBlockIterator.hasNext()) {
-            nextRead.set(true);
-            futureIo = executeRead();
-          }
-          return blockletScanner.scanBlocklet(blocksChunkHolder);
-        }
-        return null;
-      }
-    });
-  }
-
-  private Future<BlocksChunkHolder> executeRead() {
-    return executorService.submit(new Callable<BlocksChunkHolder>() {
-      @Override public BlocksChunkHolder call() throws Exception {
-        try {
-          TaskMetricsMap.getInstance().registerThreadCallback();
-          if (dataBlockIterator.hasNext()) {
-            BlocksChunkHolder blocksChunkHolder = getBlocksChunkHolder();
-            if (blocksChunkHolder != null) {
-              blockletScanner.readBlocklet(blocksChunkHolder);
-              return blocksChunkHolder;
-            }
-          }
-          return null;
-        } finally {
-          // update read bytes metrics for this thread
-          TaskMetricsMap.getInstance().updateReadBytes(Thread.currentThread().getId());
-        }
-      }
-    });
-  }
-
-  public abstract void processNextBatch(CarbonColumnarBatch columnarBatch);
-
-  /**
-   * Close the resources
-   */
-  public void close() {
-    // free the current scanned result
-    if (null != scannedResult && !scannedResult.hasNext()) {
-      scannedResult.freeMemory();
-    }
-    // free any pre-fetched memory if present
-    if (null != future) {
-      try {
-        AbstractScannedResult abstractScannedResult = future.get();
-        if (abstractScannedResult != null) {
-          abstractScannedResult.freeMemory();
-        }
-      } catch (InterruptedException | ExecutionException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java
index 3ea4e1d..f0d81ff 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/BlockletIterator.java
@@ -23,7 +23,7 @@ import org.apache.carbondata.core.datastore.DataRefNode;
 /**
  * Below class will be used to iterate over data block
  */
-public class BlockletIterator extends CarbonIterator<DataRefNode> {
+class BlockletIterator extends CarbonIterator<DataRefNode> {
   /**
    * data store block
    */
@@ -47,17 +47,18 @@ public class BlockletIterator extends CarbonIterator<DataRefNode> {
    * Constructor
    *
    * @param datablock                 first data block
-   * @param totalNumberOfBlocksToScan total number of blocks to be scanned
+   * @param totalNumberOfBlockletToScan total number of blocklets to be scanned
    */
-  public BlockletIterator(DataRefNode datablock, long totalNumberOfBlocksToScan) {
+  BlockletIterator(DataRefNode datablock, long totalNumberOfBlockletToScan) {
     this.datablock = datablock;
-    this.totalNumberOfBlocksToScan = totalNumberOfBlocksToScan;
+    this.totalNumberOfBlocksToScan = totalNumberOfBlockletToScan;
   }
 
   /**
    * is all the blocks assigned to this iterator has been processed
    */
-  @Override public boolean hasNext() {
+  @Override
+  public boolean hasNext() {
     return hasNext;
   }
 
@@ -67,7 +68,8 @@ public class BlockletIterator extends CarbonIterator<DataRefNode> {
    * @return next data block
    *
    */
-  @Override public DataRefNode next() {
+  @Override
+  public DataRefNode next() {
     // get the current blocks
     DataRefNode datablockTemp = datablock;
     // store the next data block

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


[10/49] carbondata git commit: [CARBONDATA-2099] Refactor query scan process to improve readability

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
new file mode 100644
index 0000000..fde4e55
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/DataBlockIterator.java
@@ -0,0 +1,269 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.processor;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.scan.collector.ResultCollectorFactory;
+import org.apache.carbondata.core.scan.collector.ScannedResultCollector;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
+import org.apache.carbondata.core.scan.scanner.BlockletScanner;
+import org.apache.carbondata.core.scan.scanner.impl.BlockletFilterScanner;
+import org.apache.carbondata.core.scan.scanner.impl.BlockletFullScanner;
+import org.apache.carbondata.core.stats.QueryStatisticsModel;
+import org.apache.carbondata.core.util.TaskMetricsMap;
+
+/**
+ * This abstract class provides a skeletal implementation of the
+ * Block iterator.
+ */
+public class DataBlockIterator extends CarbonIterator<List<Object[]>> {
+
+  /**
+   * iterator which will be used to iterate over blocklets
+   */
+  private BlockletIterator blockletIterator;
+
+  /**
+   * result collector which will be used to aggregate the scanned result
+   */
+  private ScannedResultCollector scannerResultAggregator;
+
+  /**
+   * processor which will be used to process the block processing can be
+   * filter processing or non filter processing
+   */
+  private BlockletScanner blockletScanner;
+
+  /**
+   * batch size of result
+   */
+  private int batchSize;
+
+  private ExecutorService executorService;
+
+  private Future<BlockletScannedResult> future;
+
+  private Future<RawBlockletColumnChunks> futureIo;
+
+  private BlockletScannedResult scannedResult;
+
+  private BlockExecutionInfo blockExecutionInfo;
+
+  private FileReader fileReader;
+
+  private AtomicBoolean nextBlock;
+
+  private AtomicBoolean nextRead;
+
+  public DataBlockIterator(BlockExecutionInfo blockExecutionInfo, FileReader fileReader,
+      int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) {
+    this.blockExecutionInfo = blockExecutionInfo;
+    this.fileReader = fileReader;
+    blockletIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
+        blockExecutionInfo.getNumberOfBlockToScan());
+    if (blockExecutionInfo.getFilterExecuterTree() != null) {
+      blockletScanner = new BlockletFilterScanner(blockExecutionInfo, queryStatisticsModel);
+    } else {
+      blockletScanner = new BlockletFullScanner(blockExecutionInfo, queryStatisticsModel);
+    }
+    this.scannerResultAggregator =
+        ResultCollectorFactory.getScannedResultCollector(blockExecutionInfo);
+    this.batchSize = batchSize;
+    this.executorService = executorService;
+    this.nextBlock = new AtomicBoolean(false);
+    this.nextRead = new AtomicBoolean(false);
+  }
+
+  @Override
+  public List<Object[]> next() {
+    List<Object[]> collectedResult = null;
+    if (updateScanner()) {
+      collectedResult = this.scannerResultAggregator.collectResultInRow(scannedResult, batchSize);
+      while (collectedResult.size() < batchSize && updateScanner()) {
+        List<Object[]> data = this.scannerResultAggregator
+            .collectResultInRow(scannedResult, batchSize - collectedResult.size());
+        collectedResult.addAll(data);
+      }
+    } else {
+      collectedResult = new ArrayList<>();
+    }
+    return collectedResult;
+  }
+
+  @Override
+  public boolean hasNext() {
+    if (scannedResult != null && scannedResult.hasNext()) {
+      return true;
+    } else {
+      if (null != scannedResult) {
+        scannedResult.freeMemory();
+      }
+      return blockletIterator.hasNext() || nextBlock.get() || nextRead.get();
+    }
+  }
+
+  /**
+   * Return true if scan result if non-empty
+   */
+  private boolean updateScanner() {
+    try {
+      if (scannedResult != null && scannedResult.hasNext()) {
+        return true;
+      } else {
+        scannedResult = processNextBlocklet();
+        while (scannedResult != null) {
+          if (scannedResult.hasNext()) {
+            return true;
+          }
+          scannedResult = processNextBlocklet();
+        }
+        nextBlock.set(false);
+        nextRead.set(false);
+        return false;
+      }
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  private BlockletScannedResult processNextBlocklet() throws Exception {
+    BlockletScannedResult result = null;
+    if (blockExecutionInfo.isPrefetchBlocklet()) {
+      if (blockletIterator.hasNext() || nextBlock.get() || nextRead.get()) {
+        if (future == null) {
+          future = scanNextBlockletAsync();
+        }
+        result = future.get();
+        nextBlock.set(false);
+        if (blockletIterator.hasNext() || nextRead.get()) {
+          nextBlock.set(true);
+          future = scanNextBlockletAsync();
+        }
+      }
+    } else {
+      if (blockletIterator.hasNext()) {
+        RawBlockletColumnChunks rawChunks = readNextBlockletColumnChunks();
+        if (rawChunks != null) {
+          result = blockletScanner.scanBlocklet(rawChunks);
+        }
+      }
+    }
+    return result;
+  }
+
+  private RawBlockletColumnChunks readNextBlockletColumnChunks() throws IOException {
+    RawBlockletColumnChunks rawBlockletColumnChunks = getNextBlockletColumnChunks();
+    if (rawBlockletColumnChunks != null) {
+      blockletScanner.readBlocklet(rawBlockletColumnChunks);
+      return rawBlockletColumnChunks;
+    }
+    return null;
+  }
+
+  private RawBlockletColumnChunks getNextBlockletColumnChunks() {
+    RawBlockletColumnChunks rawBlockletColumnChunks = null;
+    do {
+      DataRefNode dataBlock = blockletIterator.next();
+      if (dataBlock.getColumnsMaxValue() == null || blockletScanner.isScanRequired(dataBlock)) {
+        rawBlockletColumnChunks =  RawBlockletColumnChunks.newInstance(
+            blockExecutionInfo.getTotalNumberDimensionToRead(),
+            blockExecutionInfo.getTotalNumberOfMeasureToRead(), fileReader, dataBlock);
+      }
+    } while (rawBlockletColumnChunks == null && blockletIterator.hasNext());
+    return rawBlockletColumnChunks;
+  }
+
+  private Future<BlockletScannedResult> scanNextBlockletAsync() {
+    return executorService.submit(new Callable<BlockletScannedResult>() {
+      @Override public BlockletScannedResult call() throws Exception {
+        if (futureIo == null) {
+          futureIo = readNextBlockletAsync();
+        }
+        RawBlockletColumnChunks rawBlockletColumnChunks = futureIo.get();
+        futureIo = null;
+        nextRead.set(false);
+        if (rawBlockletColumnChunks != null) {
+          if (blockletIterator.hasNext()) {
+            nextRead.set(true);
+            futureIo = readNextBlockletAsync();
+          }
+          return blockletScanner.scanBlocklet(rawBlockletColumnChunks);
+        }
+        return null;
+      }
+    });
+  }
+
+  private Future<RawBlockletColumnChunks> readNextBlockletAsync() {
+    return executorService.submit(new Callable<RawBlockletColumnChunks>() {
+      @Override public RawBlockletColumnChunks call() throws Exception {
+        try {
+          TaskMetricsMap.getInstance().registerThreadCallback();
+          if (blockletIterator.hasNext()) {
+            return readNextBlockletColumnChunks();
+          } else {
+            return null;
+          }
+        } finally {
+          // update read bytes metrics for this thread
+          TaskMetricsMap.getInstance().updateReadBytes(Thread.currentThread().getId());
+        }
+      }
+    });
+  }
+
+  public void processNextBatch(CarbonColumnarBatch columnarBatch) {
+    if (updateScanner()) {
+      this.scannerResultAggregator.collectResultInColumnarBatch(scannedResult, columnarBatch);
+    }
+  }
+
+
+  /**
+   * Close the resources
+   */
+  public void close() {
+    // free the current scanned result
+    if (null != scannedResult && !scannedResult.hasNext()) {
+      scannedResult.freeMemory();
+    }
+    // free any pre-fetched memory if present
+    if (null != future) {
+      try {
+        BlockletScannedResult blockletScannedResult = future.get();
+        if (blockletScannedResult != null) {
+          blockletScannedResult.freeMemory();
+        }
+      } catch (InterruptedException | ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java
new file mode 100644
index 0000000..6b7e880
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/processor/RawBlockletColumnChunks.java
@@ -0,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.apache.carbondata.core.scan.processor;
+
+import org.apache.carbondata.core.datastore.DataRefNode;
+import org.apache.carbondata.core.datastore.FileReader;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.util.BitSetGroup;
+
+/**
+ * Contains dimension and measure raw column chunks of one blocklet
+ */
+public class RawBlockletColumnChunks {
+
+  /**
+   * dimension column data chunk
+   */
+  private DimensionRawColumnChunk[] dimensionRawColumnChunks;
+
+  /**
+   * measure column data chunk
+   */
+  private MeasureRawColumnChunk[] measureRawColumnChunks;
+
+  /**
+   * file reader which will use to read the block from file
+   */
+  private FileReader fileReader;
+
+  /**
+   * data block
+   */
+  private DataRefNode dataBlock;
+
+  private BitSetGroup bitSetGroup;
+
+  private RawBlockletColumnChunks() { }
+
+  public static RawBlockletColumnChunks newInstance(int numberOfDimensionChunk,
+      int numberOfMeasureChunk, FileReader fileReader, DataRefNode dataBlock) {
+    RawBlockletColumnChunks instance = new RawBlockletColumnChunks();
+    instance.dimensionRawColumnChunks = new DimensionRawColumnChunk[numberOfDimensionChunk];
+    instance.measureRawColumnChunks = new MeasureRawColumnChunk[numberOfMeasureChunk];
+    instance.fileReader = fileReader;
+    instance.dataBlock = dataBlock;
+    return instance;
+  }
+
+  /**
+   * @return the dimensionRawColumnChunks
+   */
+  public DimensionRawColumnChunk[] getDimensionRawColumnChunks() {
+    return dimensionRawColumnChunks;
+  }
+
+  /**
+   * @param dimensionRawColumnChunks the dimensionRawColumnChunks to set
+   */
+  public void setDimensionRawColumnChunks(DimensionRawColumnChunk[] dimensionRawColumnChunks) {
+    this.dimensionRawColumnChunks = dimensionRawColumnChunks;
+  }
+
+  /**
+   * @return the measureRawColumnChunks
+   */
+  public MeasureRawColumnChunk[] getMeasureRawColumnChunks() {
+    return measureRawColumnChunks;
+  }
+
+  /**
+   * @param measureRawColumnChunks the measureRawColumnChunks to set
+   */
+  public void setMeasureRawColumnChunks(MeasureRawColumnChunk[] measureRawColumnChunks) {
+    this.measureRawColumnChunks = measureRawColumnChunks;
+  }
+
+  /**
+   * @return the fileReader
+   */
+  public FileReader getFileReader() {
+    return fileReader;
+  }
+
+  /**
+   * @return the dataBlock
+   */
+  public DataRefNode getDataBlock() {
+    return dataBlock;
+  }
+
+  public BitSetGroup getBitSetGroup() {
+    return bitSetGroup;
+  }
+
+  public void setBitSetGroup(BitSetGroup bitSetGroup) {
+    this.bitSetGroup = bitSetGroup;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java b/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java
deleted file mode 100644
index 1c97725..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/processor/impl/DataBlockIteratorImpl.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.processor.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-
-import org.apache.carbondata.core.datastore.FileHolder;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.processor.AbstractDataBlockIterator;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.stats.QueryStatisticsModel;
-
-/**
- * Below class will be used to process the block for detail query
- */
-public class DataBlockIteratorImpl extends AbstractDataBlockIterator {
-  /**
-   * DataBlockIteratorImpl Constructor
-   *
-   * @param blockExecutionInfo execution information
-   */
-  public DataBlockIteratorImpl(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader,
-      int batchSize, QueryStatisticsModel queryStatisticsModel, ExecutorService executorService) {
-    super(blockExecutionInfo, fileReader, batchSize, queryStatisticsModel, executorService);
-  }
-
-  /**
-   * It scans the block and returns the result with @batchSize
-   *
-   * @return Result of @batchSize
-   */
-  public List<Object[]> next() {
-    List<Object[]> collectedResult = null;
-    if (updateScanner()) {
-      collectedResult = this.scannerResultAggregator.collectData(scannedResult, batchSize);
-      while (collectedResult.size() < batchSize && updateScanner()) {
-        List<Object[]> data = this.scannerResultAggregator
-            .collectData(scannedResult, batchSize - collectedResult.size());
-        collectedResult.addAll(data);
-      }
-    } else {
-      collectedResult = new ArrayList<>();
-    }
-    return collectedResult;
-  }
-
-  public void processNextBatch(CarbonColumnarBatch columnarBatch) {
-    if (updateScanner()) {
-      this.scannerResultAggregator.collectVectorBatch(scannedResult, columnarBatch);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
deleted file mode 100644
index b089fad..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/AbstractScannedResult.java
+++ /dev/null
@@ -1,698 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.core.scan.result;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.Map;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.chunk.DimensionColumnDataChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
-import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
-import org.apache.carbondata.core.datastore.page.ColumnPage;
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
-import org.apache.carbondata.core.mutate.DeleteDeltaVo;
-import org.apache.carbondata.core.mutate.TupleIdEnum;
-import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
-import org.apache.carbondata.core.scan.filter.GenericQueryType;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
-import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
-import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-
-/**
- * Scanned result class which will store and provide the result on request
- */
-public abstract class AbstractScannedResult {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractScannedResult.class.getName());
-  /**
-   * current row number
-   */
-  protected int currentRow = -1;
-
-  protected int pageCounter;
-  /**
-   * row mapping indexes
-   */
-  protected int[][] rowMapping;
-  /**
-   * key size of the fixed length column
-   */
-  private int fixedLengthKeySize;
-  /**
-   * total number of rows per page
-   */
-  private int[] numberOfRows;
-
-  /**
-   * Total number of rows.
-   */
-  private int totalNumberOfRows;
-  /**
-   * to keep track of number of rows process
-   */
-  protected int rowCounter;
-  /**
-   * dimension column data chunk
-   */
-  protected DimensionColumnDataChunk[][] dimensionDataChunks;
-
-  /**
-   * Raw dimension chunks;
-   */
-  protected DimensionRawColumnChunk[] dimRawColumnChunks;
-
-  /**
-   * Raw dimension chunks;
-   */
-  protected MeasureRawColumnChunk[] msrRawColumnChunks;
-  /**
-   * measure column data chunk
-   */
-  protected ColumnPage[][] measureDataChunks;
-  /**
-   * dictionary column block index in file
-   */
-  protected int[] dictionaryColumnBlockIndexes;
-
-  /**
-   * no dictionary column block index in file
-   */
-  protected int[] noDictionaryColumnBlockIndexes;
-
-  /**
-   * column group to is key structure info
-   * which will be used to get the key from the complete
-   * column group key
-   * For example if only one dimension of the column group is selected
-   * then from complete column group key it will be used to mask the key and
-   * get the particular column key
-   */
-  protected Map<Integer, KeyStructureInfo> columnGroupKeyStructureInfo;
-
-  /**
-   *
-   */
-  private Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
-
-  private int totalDimensionsSize;
-
-  /**
-   * blockedId which will be blockId + blocklet number in the block
-   */
-  private String blockletId;
-
-  private long rowId;
-
-  /**
-   * parent block indexes
-   */
-  private int[] complexParentBlockIndexes;
-
-  /**
-   * blockletid+pageumber to deleted reocrd map
-   */
-  private Map<String, DeleteDeltaVo> deletedRecordMap;
-
-  /**
-   * current page delete delta vo
-   */
-  private DeleteDeltaVo currentDeleteDeltaVo;
-
-  /**
-   * actual blocklet number
-   */
-  private String blockletNumber;
-
-  public AbstractScannedResult(BlockExecutionInfo blockExecutionInfo) {
-    this.fixedLengthKeySize = blockExecutionInfo.getFixedLengthKeySize();
-    this.noDictionaryColumnBlockIndexes = blockExecutionInfo.getNoDictionaryBlockIndexes();
-    this.dictionaryColumnBlockIndexes = blockExecutionInfo.getDictionaryColumnBlockIndex();
-    this.columnGroupKeyStructureInfo = blockExecutionInfo.getColumnGroupToKeyStructureInfo();
-    this.complexParentIndexToQueryMap = blockExecutionInfo.getComlexDimensionInfoMap();
-    this.complexParentBlockIndexes = blockExecutionInfo.getComplexColumnParentBlockIndexes();
-    this.totalDimensionsSize = blockExecutionInfo.getQueryDimensions().length;
-    this.deletedRecordMap = blockExecutionInfo.getDeletedRecordsMap();
-  }
-
-  /**
-   * Below method will be used to set the dimension chunks
-   * which will be used to create a row
-   *
-   * @param dataChunks dimension chunks used in query
-   */
-  public void setDimensionChunks(DimensionColumnDataChunk[][] dataChunks) {
-    this.dimensionDataChunks = dataChunks;
-  }
-
-  /**
-   * Below method will be used to set the measure column chunks
-   *
-   * @param measureDataChunks measure data chunks
-   */
-  public void setMeasureChunks(ColumnPage[][] measureDataChunks) {
-    this.measureDataChunks = measureDataChunks;
-  }
-
-  public void setDimRawColumnChunks(DimensionRawColumnChunk[] dimRawColumnChunks) {
-    this.dimRawColumnChunks = dimRawColumnChunks;
-  }
-
-  public void setMsrRawColumnChunks(MeasureRawColumnChunk[] msrRawColumnChunks) {
-    this.msrRawColumnChunks = msrRawColumnChunks;
-  }
-
-  /**
-   * Below method will be used to get the chunk based in measure ordinal
-   *
-   * @param ordinal measure ordinal
-   * @return measure column chunk
-   */
-  public ColumnPage getMeasureChunk(int ordinal) {
-    return measureDataChunks[ordinal][pageCounter];
-  }
-
-  /**
-   * Below method will be used to get the key for all the dictionary dimensions
-   * which is present in the query
-   *
-   * @param rowId row id selected after scanning
-   * @return return the dictionary key
-   */
-  protected byte[] getDictionaryKeyArray(int rowId) {
-    byte[] completeKey = new byte[fixedLengthKeySize];
-    int offset = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      offset += dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillChunkData(completeKey, offset, rowId,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
-    }
-    rowCounter++;
-    return completeKey;
-  }
-
-  /**
-   * Below method will be used to get the key for all the dictionary dimensions
-   * in integer array format which is present in the query
-   *
-   * @param rowId row id selected after scanning
-   * @return return the dictionary key
-   */
-  protected int[] getDictionaryKeyIntegerArray(int rowId) {
-    int[] completeKey = new int[totalDimensionsSize];
-    int column = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(rowId, column, completeKey,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
-    }
-    rowCounter++;
-    return completeKey;
-  }
-
-  /**
-   * Fill the column data of dictionary to vector
-   */
-  public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
-    int column = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(vectorInfo, column,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
-    }
-  }
-
-  /**
-   * Fill the column data to vector
-   */
-  public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
-    int column = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(vectorInfo, column,
-              columnGroupKeyStructureInfo.get(noDictionaryColumnBlockIndexes[i]));
-    }
-  }
-
-  /**
-   * Fill the measure column data to vector
-   */
-  public void fillColumnarMeasureBatch(ColumnVectorInfo[] vectorInfo, int[] measuresOrdinal) {
-    for (int i = 0; i < measuresOrdinal.length; i++) {
-      vectorInfo[i].measureVectorFiller
-          .fillMeasureVector(measureDataChunks[measuresOrdinal[i]][pageCounter], vectorInfo[i]);
-    }
-  }
-
-  public void fillColumnarComplexBatch(ColumnVectorInfo[] vectorInfos) {
-    for (int i = 0; i < vectorInfos.length; i++) {
-      int offset = vectorInfos[i].offset;
-      int len = offset + vectorInfos[i].size;
-      int vectorOffset = vectorInfos[i].vectorOffset;
-      CarbonColumnVector vector = vectorInfos[i].vector;
-      for (int j = offset; j < len; j++) {
-        ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
-        DataOutputStream dataOutput = new DataOutputStream(byteStream);
-        try {
-          vectorInfos[i].genericQueryType
-              .parseBlocksAndReturnComplexColumnByteArray(dimRawColumnChunks,
-                  rowMapping == null ? j : rowMapping[pageCounter][j], pageCounter, dataOutput);
-          Object data = vectorInfos[i].genericQueryType
-              .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(byteStream.toByteArray()));
-          vector.putObject(vectorOffset++, data);
-        } catch (IOException e) {
-          LOGGER.error(e);
-        } finally {
-          CarbonUtil.closeStreams(dataOutput);
-          CarbonUtil.closeStreams(byteStream);
-        }
-      }
-    }
-  }
-
-  /**
-   * Fill the column data to vector
-   */
-  public void fillColumnarImplicitBatch(ColumnVectorInfo[] vectorInfo) {
-    for (int i = 0; i < vectorInfo.length; i++) {
-      ColumnVectorInfo columnVectorInfo = vectorInfo[i];
-      CarbonColumnVector vector = columnVectorInfo.vector;
-      int offset = columnVectorInfo.offset;
-      int vectorOffset = columnVectorInfo.vectorOffset;
-      int len = offset + columnVectorInfo.size;
-      for (int j = offset; j < len; j++) {
-        // Considering only String case now as we support only
-        String data = getBlockletId();
-        if (CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID
-            .equals(columnVectorInfo.dimension.getColumnName())) {
-          data = data + CarbonCommonConstants.FILE_SEPARATOR + pageCounter
-              + CarbonCommonConstants.FILE_SEPARATOR + (rowMapping == null ?
-              j :
-              rowMapping[pageCounter][j]);
-        }
-        vector.putBytes(vectorOffset++,
-            data.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-      }
-    }
-  }
-
-  /**
-   * Just increment the counter incase of query only on measures.
-   */
-  public void incrementCounter() {
-    rowCounter++;
-    currentRow++;
-  }
-
-  /**
-   * Just increment the page counter and reset the remaining counters.
-   */
-  public void incrementPageCounter() {
-    rowCounter = 0;
-    currentRow = -1;
-    pageCounter++;
-    fillDataChunks();
-    if (null != deletedRecordMap) {
-      currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + "_" + pageCounter);
-    }
-  }
-
-  /**
-   * This case is used only in case of compaction, since it does not use filter flow.
-   */
-  public void fillDataChunks() {
-    freeDataChunkMemory();
-    if (pageCounter >= numberOfRows.length) {
-      return;
-    }
-    for (int i = 0; i < dimensionDataChunks.length; i++) {
-      if (dimensionDataChunks[i][pageCounter] == null && dimRawColumnChunks[i] != null) {
-        dimensionDataChunks[i][pageCounter] =
-            dimRawColumnChunks[i].convertToDimColDataChunkWithOutCache(pageCounter);
-      }
-    }
-
-    for (int i = 0; i < measureDataChunks.length; i++) {
-      if (measureDataChunks[i][pageCounter] == null && msrRawColumnChunks[i] != null) {
-        measureDataChunks[i][pageCounter] =
-            msrRawColumnChunks[i].convertToColumnPageWithOutCache(pageCounter);
-      }
-    }
-  }
-
-  // free the memory for the last page chunk
-  private void freeDataChunkMemory() {
-    for (int i = 0; i < dimensionDataChunks.length; i++) {
-      if (pageCounter > 0 && dimensionDataChunks[i][pageCounter - 1] != null) {
-        dimensionDataChunks[i][pageCounter - 1].freeMemory();
-        dimensionDataChunks[i][pageCounter - 1] = null;
-      }
-    }
-    for (int i = 0; i < measureDataChunks.length; i++) {
-      if (pageCounter > 0 && measureDataChunks[i][pageCounter - 1] != null) {
-        measureDataChunks[i][pageCounter - 1].freeMemory();
-        measureDataChunks[i][pageCounter - 1] = null;
-      }
-    }
-  }
-
-  public int numberOfpages() {
-    return numberOfRows.length;
-  }
-
-  /**
-   * Get total rows in the current page
-   *
-   * @return
-   */
-  public int getCurrentPageRowCount() {
-    return numberOfRows[pageCounter];
-  }
-
-  public int getCurrentPageCounter() {
-    return pageCounter;
-  }
-
-  /**
-   * increment the counter.
-   */
-  public void setRowCounter(int rowCounter) {
-    this.rowCounter = rowCounter;
-  }
-
-  /**
-   * Below method will be used to get the dimension data based on dimension
-   * ordinal and index
-   *
-   * @param dimOrdinal dimension ordinal present in the query
-   * @param rowId      row index
-   * @return dimension data based on row id
-   */
-  protected byte[] getDimensionData(int dimOrdinal, int rowId) {
-    return dimensionDataChunks[dimOrdinal][pageCounter].getChunkData(rowId);
-  }
-
-  /**
-   * Below method will be used to get the dimension key array
-   * for all the no dictionary dimension present in the query
-   *
-   * @param rowId row number
-   * @return no dictionary keys for all no dictionary dimension
-   */
-  protected byte[][] getNoDictionaryKeyArray(int rowId) {
-    byte[][] noDictionaryColumnsKeys = new byte[noDictionaryColumnBlockIndexes.length][];
-    int position = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      noDictionaryColumnsKeys[position++] =
-          dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter].getChunkData(rowId);
-    }
-    return noDictionaryColumnsKeys;
-  }
-
-  /**
-   * Below method will be used to get the dimension key array
-   * for all the no dictionary dimension present in the query
-   *
-   * @param rowId row number
-   * @return no dictionary keys for all no dictionary dimension
-   */
-  protected String[] getNoDictionaryKeyStringArray(int rowId) {
-    String[] noDictionaryColumnsKeys = new String[noDictionaryColumnBlockIndexes.length];
-    int position = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      noDictionaryColumnsKeys[position++] = new String(
-          dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter].getChunkData(rowId),
-          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-    }
-    return noDictionaryColumnsKeys;
-  }
-
-  /**
-   * @return blockletId
-   */
-  public String getBlockletId() {
-    return blockletId;
-  }
-
-  /**
-   * @param blockletId
-   */
-  public void setBlockletId(String blockletId) {
-    this.blockletId = CarbonTablePath.getShortBlockId(blockletId);
-    blockletNumber = CarbonUpdateUtil.getRequiredFieldFromTID(blockletId, TupleIdEnum.BLOCKLET_ID);
-    // if deleted recors map is present for this block
-    // then get the first page deleted vo
-    if (null != deletedRecordMap) {
-      currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + '_' + pageCounter);
-    }
-  }
-
-  /**
-   * @return blockletId
-   */
-  public long getRowId() {
-    return rowId;
-  }
-
-  /**
-   * @param rowId
-   */
-  public void setRowId(long rowId) {
-    this.rowId = rowId;
-  }
-
-  /**
-   * Below method will be used to get the complex type keys array based
-   * on row id for all the complex type dimension selected in query
-   *
-   * @param rowId row number
-   * @return complex type key array for all the complex dimension selected in query
-   */
-  protected byte[][] getComplexTypeKeyArray(int rowId) {
-    byte[][] complexTypeData = new byte[complexParentBlockIndexes.length][];
-    for (int i = 0; i < complexTypeData.length; i++) {
-      GenericQueryType genericQueryType =
-          complexParentIndexToQueryMap.get(complexParentBlockIndexes[i]);
-      ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
-      DataOutputStream dataOutput = new DataOutputStream(byteStream);
-      try {
-        genericQueryType
-            .parseBlocksAndReturnComplexColumnByteArray(dimRawColumnChunks, rowId, pageCounter,
-                dataOutput);
-        complexTypeData[i] = byteStream.toByteArray();
-      } catch (IOException e) {
-        LOGGER.error(e);
-      } finally {
-        CarbonUtil.closeStreams(dataOutput);
-        CarbonUtil.closeStreams(byteStream);
-      }
-    }
-    return complexTypeData;
-  }
-
-  /**
-   * @return return the total number of row after scanning
-   */
-  public int numberOfOutputRows() {
-    return this.totalNumberOfRows;
-  }
-
-  /**
-   * to check whether any more row is present in the result
-   *
-   * @return
-   */
-  public boolean hasNext() {
-    if (pageCounter < numberOfRows.length && rowCounter < this.numberOfRows[pageCounter]) {
-      return true;
-    } else if (pageCounter < numberOfRows.length) {
-      pageCounter++;
-      fillDataChunks();
-      rowCounter = 0;
-      currentRow = -1;
-      if (null != deletedRecordMap) {
-        currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + "_" + pageCounter);
-      }
-      return hasNext();
-    }
-    return false;
-  }
-
-  /**
-   * Below method will be used to free the occupied memory
-   */
-  public void freeMemory() {
-    // first free the dimension chunks
-    if (null != dimensionDataChunks) {
-      for (int i = 0; i < dimensionDataChunks.length; i++) {
-        if (null != dimensionDataChunks[i]) {
-          for (int j = 0; j < dimensionDataChunks[i].length; j++) {
-            if (null != dimensionDataChunks[i][j]) {
-              dimensionDataChunks[i][j].freeMemory();
-            }
-          }
-        }
-      }
-    }
-    // free the measure data chunks
-    if (null != measureDataChunks) {
-      for (int i = 0; i < measureDataChunks.length; i++) {
-        if (null != measureDataChunks[i]) {
-          for (int j = 0; j < measureDataChunks[i].length; j++) {
-            if (null != measureDataChunks[i][j]) {
-              measureDataChunks[i][j].freeMemory();
-            }
-          }
-        }
-      }
-    }
-    // free the raw chunks
-    if (null != dimRawColumnChunks) {
-      for (int i = 0; i < dimRawColumnChunks.length; i++) {
-        if (null != dimRawColumnChunks[i]) {
-          dimRawColumnChunks[i].freeMemory();
-        }
-      }
-    }
-  }
-
-  /**
-   * As this class will be a flyweight object so
-   * for one block all the blocklet scanning will use same result object
-   * in that case we need to reset the counter to zero so
-   * for new result it will give the result from zero
-   */
-  public void reset() {
-    rowCounter = 0;
-    currentRow = -1;
-    pageCounter = 0;
-  }
-
-  /**
-   * @param numberOfRows set total of number rows valid after scanning
-   */
-  public void setNumberOfRows(int[] numberOfRows) {
-    this.numberOfRows = numberOfRows;
-
-    for (int count : numberOfRows) {
-      totalNumberOfRows += count;
-    }
-  }
-
-  /**
-   * After applying filter it will return the  bit set with the valid row indexes
-   * so below method will be used to set the row indexes
-   *
-   * @param indexes
-   */
-  public void setIndexes(int[][] indexes) {
-    this.rowMapping = indexes;
-  }
-
-  public int getRowCounter() {
-    return rowCounter;
-  }
-
-  /**
-   * will return the current valid row id
-   *
-   * @return valid row id
-   */
-  public abstract int getCurrentRowId();
-
-  /**
-   * @return dictionary key array for all the dictionary dimension
-   * selected in query
-   */
-  public abstract byte[] getDictionaryKeyArray();
-
-  /**
-   * @return dictionary key array for all the dictionary dimension in integer array forat
-   * selected in query
-   */
-  public abstract int[] getDictionaryKeyIntegerArray();
-
-  /**
-   * Below method will be used to get the complex type key array
-   *
-   * @return complex type key array
-   */
-  public abstract byte[][] getComplexTypeKeyArray();
-
-  /**
-   * Below method will be used to get the no dictionary key
-   * array for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  public abstract byte[][] getNoDictionaryKeyArray();
-
-  /**
-   * Below method will be used to get the no dictionary key
-   * array in string array format for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  public abstract String[] getNoDictionaryKeyStringArray();
-
-  /**
-   * Mark the filtered rows in columnar batch. These rows will not be added to vector batches later.
-   * @param columnarBatch
-   * @param startRow
-   * @param size
-   * @param vectorOffset
-   */
-  public int markFilteredRows(CarbonColumnarBatch columnarBatch, int startRow, int size,
-      int vectorOffset) {
-    int rowsFiltered = 0;
-    if (currentDeleteDeltaVo != null) {
-      int len = startRow + size;
-      for (int i = startRow; i < len; i++) {
-        int rowId = rowMapping != null ? rowMapping[pageCounter][i] : i;
-        if (currentDeleteDeltaVo.containsRow(rowId)) {
-          columnarBatch.markFiltered(vectorOffset);
-          rowsFiltered++;
-        }
-        vectorOffset++;
-      }
-    }
-    return rowsFiltered;
-  }
-
-  /**
-   * Below method will be used to check row got deleted
-   *
-   * @param rowId
-   * @return is present in deleted row
-   */
-  public boolean containsDeletedRow(int rowId) {
-    if (null != currentDeleteDeltaVo) {
-      return currentDeleteDeltaVo.containsRow(rowId);
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java
deleted file mode 100644
index 56ca2ac..0000000
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/BatchResult.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.core.scan.result;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import org.apache.carbondata.common.CarbonIterator;
-
-/**
- * Below class holds the query result
- */
-public class BatchResult extends CarbonIterator<Object[]> {
-
-  /**
-   * list of keys
-   */
-  protected List<Object[]> rows;
-
-  /**
-   * counter to check whether all the records are processed or not
-   */
-  protected int counter;
-
-  public BatchResult() {
-    this.rows = new ArrayList<>();
-  }
-
-  /**
-   * Below method will be used to get the rows
-   *
-   * @return
-   */
-  public List<Object[]> getRows() {
-    return rows;
-  }
-
-  /**
-   * Below method will be used to get the set the values
-   *
-   * @param rows
-   */
-  public void setRows(List<Object[]> rows) {
-    this.rows = rows;
-  }
-
-  /**
-   * This method will return one row at a time based on the counter given.
-   * @param counter
-   * @return
-   */
-  public Object[] getRawRow(int counter) {
-    return rows.get(counter);
-  }
-
-  /**
-   * For getting the total size.
-   * @return
-   */
-  public int getSize() {
-    return rows.size();
-  }
-
-
-  /**
-   * Returns {@code true} if the iteration has more elements.
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return counter < rows.size();
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public Object[] next() {
-    if (!hasNext()) {
-      throw new NoSuchElementException();
-    }
-    Object[] row = rows.get(counter);
-    counter++;
-    return row;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
new file mode 100644
index 0000000..29404b4
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/BlockletScannedResult.java
@@ -0,0 +1,618 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.core.scan.result;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.Map;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.chunk.DimensionColumnPage;
+import org.apache.carbondata.core.datastore.chunk.impl.DimensionRawColumnChunk;
+import org.apache.carbondata.core.datastore.chunk.impl.MeasureRawColumnChunk;
+import org.apache.carbondata.core.datastore.page.ColumnPage;
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
+import org.apache.carbondata.core.mutate.DeleteDeltaVo;
+import org.apache.carbondata.core.mutate.TupleIdEnum;
+import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
+import org.apache.carbondata.core.scan.executor.infos.KeyStructureInfo;
+import org.apache.carbondata.core.scan.filter.GenericQueryType;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnVector;
+import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
+import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+/**
+ * Scanned result class which will store and provide the result on request
+ */
+public abstract class BlockletScannedResult {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(BlockletScannedResult.class.getName());
+  /**
+   * current row number
+   */
+  protected int currentRow = -1;
+
+  protected int pageCounter;
+  /**
+   * matched rowId for each page
+   */
+  protected int[][] pageFilteredRowId;
+  /**
+   * key size of the fixed length column
+   */
+  private int fixedLengthKeySize;
+  /**
+   * total number of filtered rows for each page
+   */
+  private int[] pageFilteredRowCount;
+
+  /**
+   * to keep track of number of rows process
+   */
+  protected int rowCounter;
+  /**
+   * dimension column data chunk
+   */
+  protected DimensionColumnPage[][] dimensionColumnPages;
+
+  /**
+   * Raw dimension chunks;
+   */
+  protected DimensionRawColumnChunk[] dimRawColumnChunks;
+
+  /**
+   * Raw dimension chunks;
+   */
+  protected MeasureRawColumnChunk[] msrRawColumnChunks;
+  /**
+   * measure column data chunk
+   */
+  protected ColumnPage[][] measureColumnPages;
+  /**
+   * dictionary column block index in file
+   */
+  protected int[] dictionaryColumnChunkIndexes;
+
+  /**
+   * no dictionary column chunk index in file
+   */
+  protected int[] noDictionaryColumnChunkIndexes;
+
+  /**
+   * column group to is key structure info
+   * which will be used to get the key from the complete
+   * column group key
+   * For example if only one dimension of the column group is selected
+   * then from complete column group key it will be used to mask the key and
+   * get the particular column key
+   */
+  protected Map<Integer, KeyStructureInfo> columnGroupKeyStructureInfo;
+
+  /**
+   *
+   */
+  private Map<Integer, GenericQueryType> complexParentIndexToQueryMap;
+
+  private int totalDimensionsSize;
+
+  /**
+   * blockedId which will be blockId + blocklet number in the block
+   */
+  private String blockletId;
+
+  /**
+   * parent block indexes
+   */
+  private int[] complexParentBlockIndexes;
+
+  /**
+   * blockletid+pageumber to deleted reocrd map
+   */
+  private Map<String, DeleteDeltaVo> deletedRecordMap;
+
+  /**
+   * current page delete delta vo
+   */
+  private DeleteDeltaVo currentDeleteDeltaVo;
+
+  /**
+   * actual blocklet number
+   */
+  private String blockletNumber;
+
+  public BlockletScannedResult(BlockExecutionInfo blockExecutionInfo) {
+    this.fixedLengthKeySize = blockExecutionInfo.getFixedLengthKeySize();
+    this.noDictionaryColumnChunkIndexes = blockExecutionInfo.getNoDictionaryColumnChunkIndexes();
+    this.dictionaryColumnChunkIndexes = blockExecutionInfo.getDictionaryColumnChunkIndex();
+    this.columnGroupKeyStructureInfo = blockExecutionInfo.getColumnGroupToKeyStructureInfo();
+    this.complexParentIndexToQueryMap = blockExecutionInfo.getComlexDimensionInfoMap();
+    this.complexParentBlockIndexes = blockExecutionInfo.getComplexColumnParentBlockIndexes();
+    this.totalDimensionsSize = blockExecutionInfo.getProjectionDimensions().length;
+    this.deletedRecordMap = blockExecutionInfo.getDeletedRecordsMap();
+  }
+
+  /**
+   * Below method will be used to set the dimension chunks
+   * which will be used to create a row
+   *
+   * @param columnPages dimension chunks used in query
+   */
+  public void setDimensionColumnPages(DimensionColumnPage[][] columnPages) {
+    this.dimensionColumnPages = columnPages;
+  }
+
+  /**
+   * Below method will be used to set the measure column chunks
+   *
+   * @param columnPages measure data chunks
+   */
+  public void setMeasureColumnPages(ColumnPage[][] columnPages) {
+    this.measureColumnPages = columnPages;
+  }
+
+  public void setDimRawColumnChunks(DimensionRawColumnChunk[] dimRawColumnChunks) {
+    this.dimRawColumnChunks = dimRawColumnChunks;
+  }
+
+  public void setMsrRawColumnChunks(MeasureRawColumnChunk[] msrRawColumnChunks) {
+    this.msrRawColumnChunks = msrRawColumnChunks;
+  }
+
+  /**
+   * Below method will be used to get the chunk based in measure ordinal
+   *
+   * @param ordinal measure ordinal
+   * @return measure column chunk
+   */
+  public ColumnPage getMeasureChunk(int ordinal) {
+    return measureColumnPages[ordinal][pageCounter];
+  }
+
+  /**
+   * Below method will be used to get the key for all the dictionary dimensions
+   * which is present in the query
+   *
+   * @param rowId row id selected after scanning
+   * @return return the dictionary key
+   */
+  protected byte[] getDictionaryKeyArray(int rowId) {
+    byte[] completeKey = new byte[fixedLengthKeySize];
+    int offset = 0;
+    for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) {
+      offset += dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter].fillRawData(
+          rowId, offset, completeKey,
+          columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i]));
+    }
+    rowCounter++;
+    return completeKey;
+  }
+
+  /**
+   * Below method will be used to get the key for all the dictionary dimensions
+   * in integer array format which is present in the query
+   *
+   * @param rowId row id selected after scanning
+   * @return return the dictionary key
+   */
+  protected int[] getDictionaryKeyIntegerArray(int rowId) {
+    int[] completeKey = new int[totalDimensionsSize];
+    int column = 0;
+    for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) {
+      column = dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter]
+          .fillSurrogateKey(rowId, column, completeKey,
+              columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i]));
+    }
+    rowCounter++;
+    return completeKey;
+  }
+
+  /**
+   * Fill the column data of dictionary to vector
+   */
+  public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
+    int column = 0;
+    for (int i = 0; i < this.dictionaryColumnChunkIndexes.length; i++) {
+      column = dimensionColumnPages[dictionaryColumnChunkIndexes[i]][pageCounter]
+          .fillVector(vectorInfo, column,
+              columnGroupKeyStructureInfo.get(dictionaryColumnChunkIndexes[i]));
+    }
+  }
+
+  /**
+   * Fill the column data to vector
+   */
+  public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
+    int column = 0;
+    for (int i = 0; i < this.noDictionaryColumnChunkIndexes.length; i++) {
+      column = dimensionColumnPages[noDictionaryColumnChunkIndexes[i]][pageCounter]
+          .fillVector(vectorInfo, column,
+              columnGroupKeyStructureInfo.get(noDictionaryColumnChunkIndexes[i]));
+    }
+  }
+
+  /**
+   * Fill the measure column data to vector
+   */
+  public void fillColumnarMeasureBatch(ColumnVectorInfo[] vectorInfo, int[] measuresOrdinal) {
+    for (int i = 0; i < measuresOrdinal.length; i++) {
+      vectorInfo[i].measureVectorFiller
+          .fillMeasureVector(measureColumnPages[measuresOrdinal[i]][pageCounter], vectorInfo[i]);
+    }
+  }
+
+  public void fillColumnarComplexBatch(ColumnVectorInfo[] vectorInfos) {
+    for (int i = 0; i < vectorInfos.length; i++) {
+      int offset = vectorInfos[i].offset;
+      int len = offset + vectorInfos[i].size;
+      int vectorOffset = vectorInfos[i].vectorOffset;
+      CarbonColumnVector vector = vectorInfos[i].vector;
+      for (int j = offset; j < len; j++) {
+        ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+        DataOutputStream dataOutput = new DataOutputStream(byteStream);
+        try {
+          vectorInfos[i].genericQueryType.parseBlocksAndReturnComplexColumnByteArray(
+              dimRawColumnChunks,
+              pageFilteredRowId == null ? j : pageFilteredRowId[pageCounter][j], pageCounter,
+              dataOutput);
+          Object data = vectorInfos[i].genericQueryType
+              .getDataBasedOnDataTypeFromSurrogates(ByteBuffer.wrap(byteStream.toByteArray()));
+          vector.putObject(vectorOffset++, data);
+        } catch (IOException e) {
+          LOGGER.error(e);
+        } finally {
+          CarbonUtil.closeStreams(dataOutput);
+          CarbonUtil.closeStreams(byteStream);
+        }
+      }
+    }
+  }
+
+  /**
+   * Fill the column data to vector
+   */
+  public void fillColumnarImplicitBatch(ColumnVectorInfo[] vectorInfo) {
+    for (int i = 0; i < vectorInfo.length; i++) {
+      ColumnVectorInfo columnVectorInfo = vectorInfo[i];
+      CarbonColumnVector vector = columnVectorInfo.vector;
+      int offset = columnVectorInfo.offset;
+      int vectorOffset = columnVectorInfo.vectorOffset;
+      int len = offset + columnVectorInfo.size;
+      for (int j = offset; j < len; j++) {
+        // Considering only String case now as we support only
+        String data = getBlockletId();
+        if (CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID
+            .equals(columnVectorInfo.dimension.getColumnName())) {
+          data = data + CarbonCommonConstants.FILE_SEPARATOR + pageCounter
+              + CarbonCommonConstants.FILE_SEPARATOR + (pageFilteredRowId == null ?
+              j :
+              pageFilteredRowId[pageCounter][j]);
+        }
+        vector.putBytes(vectorOffset++,
+            data.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+      }
+    }
+  }
+
+  /**
+   * Just increment the counter incase of query only on measures.
+   */
+  public void incrementCounter() {
+    rowCounter++;
+    currentRow++;
+  }
+
+  /**
+   * Just increment the page counter and reset the remaining counters.
+   */
+  public void incrementPageCounter() {
+    rowCounter = 0;
+    currentRow = -1;
+    pageCounter++;
+    fillDataChunks();
+    if (null != deletedRecordMap) {
+      currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + "_" + pageCounter);
+    }
+  }
+
+  /**
+   * This case is used only in case of compaction, since it does not use filter flow.
+   */
+  public void fillDataChunks() {
+    freeDataChunkMemory();
+    if (pageCounter >= pageFilteredRowCount.length) {
+      return;
+    }
+    for (int i = 0; i < dimensionColumnPages.length; i++) {
+      if (dimensionColumnPages[i][pageCounter] == null && dimRawColumnChunks[i] != null) {
+        dimensionColumnPages[i][pageCounter] =
+            dimRawColumnChunks[i].convertToDimColDataChunkWithOutCache(pageCounter);
+      }
+    }
+
+    for (int i = 0; i < measureColumnPages.length; i++) {
+      if (measureColumnPages[i][pageCounter] == null && msrRawColumnChunks[i] != null) {
+        measureColumnPages[i][pageCounter] =
+            msrRawColumnChunks[i].convertToColumnPageWithOutCache(pageCounter);
+      }
+    }
+  }
+
+  // free the memory for the last page chunk
+  private void freeDataChunkMemory() {
+    for (int i = 0; i < dimensionColumnPages.length; i++) {
+      if (pageCounter > 0 && dimensionColumnPages[i][pageCounter - 1] != null) {
+        dimensionColumnPages[i][pageCounter - 1].freeMemory();
+        dimensionColumnPages[i][pageCounter - 1] = null;
+      }
+    }
+    for (int i = 0; i < measureColumnPages.length; i++) {
+      if (pageCounter > 0 && measureColumnPages[i][pageCounter - 1] != null) {
+        measureColumnPages[i][pageCounter - 1].freeMemory();
+        measureColumnPages[i][pageCounter - 1] = null;
+      }
+    }
+  }
+
+  public int numberOfpages() {
+    return pageFilteredRowCount.length;
+  }
+
+  /**
+   * Get total rows in the current page
+   *
+   * @return
+   */
+  public int getCurrentPageRowCount() {
+    return pageFilteredRowCount[pageCounter];
+  }
+
+  public int getCurrentPageCounter() {
+    return pageCounter;
+  }
+
+  /**
+   * increment the counter.
+   */
+  public void setRowCounter(int rowCounter) {
+    this.rowCounter = rowCounter;
+  }
+
+  /**
+   * Below method will be used to get the dimension key array
+   * for all the no dictionary dimension present in the query
+   *
+   * @param rowId row number
+   * @return no dictionary keys for all no dictionary dimension
+   */
+  protected byte[][] getNoDictionaryKeyArray(int rowId) {
+    byte[][] noDictionaryColumnsKeys = new byte[noDictionaryColumnChunkIndexes.length][];
+    int position = 0;
+    for (int i = 0; i < this.noDictionaryColumnChunkIndexes.length; i++) {
+      noDictionaryColumnsKeys[position++] =
+          dimensionColumnPages[noDictionaryColumnChunkIndexes[i]][pageCounter].getChunkData(rowId);
+    }
+    return noDictionaryColumnsKeys;
+  }
+
+  /**
+   * @return blockletId
+   */
+  public String getBlockletId() {
+    return blockletId;
+  }
+
+  /**
+   * Set blocklet id, which looks like
+   * "Part0/Segment_0/part-0-0_batchno0-0-1517155583332.carbondata/0"
+   */
+  public void setBlockletId(String blockletId) {
+    this.blockletId = CarbonTablePath.getShortBlockId(blockletId);
+    blockletNumber = CarbonUpdateUtil.getRequiredFieldFromTID(blockletId, TupleIdEnum.BLOCKLET_ID);
+    // if deleted recors map is present for this block
+    // then get the first page deleted vo
+    if (null != deletedRecordMap) {
+      currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + '_' + pageCounter);
+    }
+  }
+
+  /**
+   * Below method will be used to get the complex type keys array based
+   * on row id for all the complex type dimension selected in query
+   *
+   * @param rowId row number
+   * @return complex type key array for all the complex dimension selected in query
+   */
+  protected byte[][] getComplexTypeKeyArray(int rowId) {
+    byte[][] complexTypeData = new byte[complexParentBlockIndexes.length][];
+    for (int i = 0; i < complexTypeData.length; i++) {
+      GenericQueryType genericQueryType =
+          complexParentIndexToQueryMap.get(complexParentBlockIndexes[i]);
+      ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+      DataOutputStream dataOutput = new DataOutputStream(byteStream);
+      try {
+        genericQueryType
+            .parseBlocksAndReturnComplexColumnByteArray(dimRawColumnChunks, rowId, pageCounter,
+                dataOutput);
+        complexTypeData[i] = byteStream.toByteArray();
+      } catch (IOException e) {
+        LOGGER.error(e);
+      } finally {
+        CarbonUtil.closeStreams(dataOutput);
+        CarbonUtil.closeStreams(byteStream);
+      }
+    }
+    return complexTypeData;
+  }
+
+  /**
+   * to check whether any more row is present in the result
+   *
+   * @return
+   */
+  public boolean hasNext() {
+    if (pageCounter
+        < pageFilteredRowCount.length && rowCounter < this.pageFilteredRowCount[pageCounter]) {
+      return true;
+    } else if (pageCounter < pageFilteredRowCount.length) {
+      pageCounter++;
+      fillDataChunks();
+      rowCounter = 0;
+      currentRow = -1;
+      if (null != deletedRecordMap) {
+        currentDeleteDeltaVo = deletedRecordMap.get(blockletNumber + "_" + pageCounter);
+      }
+      return hasNext();
+    }
+    return false;
+  }
+
+  /**
+   * Below method will be used to free the occupied memory
+   */
+  public void freeMemory() {
+    // first free the dimension chunks
+    if (null != dimensionColumnPages) {
+      for (int i = 0; i < dimensionColumnPages.length; i++) {
+        if (null != dimensionColumnPages[i]) {
+          for (int j = 0; j < dimensionColumnPages[i].length; j++) {
+            if (null != dimensionColumnPages[i][j]) {
+              dimensionColumnPages[i][j].freeMemory();
+            }
+          }
+        }
+      }
+    }
+    // free the measure data chunks
+    if (null != measureColumnPages) {
+      for (int i = 0; i < measureColumnPages.length; i++) {
+        if (null != measureColumnPages[i]) {
+          for (int j = 0; j < measureColumnPages[i].length; j++) {
+            if (null != measureColumnPages[i][j]) {
+              measureColumnPages[i][j].freeMemory();
+            }
+          }
+        }
+      }
+    }
+    // free the raw chunks
+    if (null != dimRawColumnChunks) {
+      for (int i = 0; i < dimRawColumnChunks.length; i++) {
+        if (null != dimRawColumnChunks[i]) {
+          dimRawColumnChunks[i].freeMemory();
+        }
+      }
+    }
+  }
+
+  /**
+   * @param pageFilteredRowCount set total of number rows valid after scanning
+   */
+  public void setPageFilteredRowCount(int[] pageFilteredRowCount) {
+    this.pageFilteredRowCount = pageFilteredRowCount;
+  }
+
+  /**
+   * After applying filter it will return the  bit set with the valid row indexes
+   * so below method will be used to set the row indexes
+   */
+  public void setPageFilteredRowId(int[][] pageFilteredRowId) {
+    this.pageFilteredRowId = pageFilteredRowId;
+  }
+
+  public int getRowCounter() {
+    return rowCounter;
+  }
+
+  /**
+   * will return the current valid row id
+   *
+   * @return valid row id
+   */
+  public abstract int getCurrentRowId();
+
+  /**
+   * @return dictionary key array for all the dictionary dimension
+   * selected in query
+   */
+  public abstract byte[] getDictionaryKeyArray();
+
+  /**
+   * @return dictionary key array for all the dictionary dimension in integer array forat
+   * selected in query
+   */
+  public abstract int[] getDictionaryKeyIntegerArray();
+
+  /**
+   * Below method will be used to get the complex type key array
+   *
+   * @return complex type key array
+   */
+  public abstract byte[][] getComplexTypeKeyArray();
+
+  /**
+   * Below method will be used to get the no dictionary key
+   * array for all the no dictionary dimension selected in query
+   *
+   * @return no dictionary key array for all the no dictionary dimension
+   */
+  public abstract byte[][] getNoDictionaryKeyArray();
+
+  /**
+   * Mark the filtered rows in columnar batch. These rows will not be added to vector batches later.
+   * @param columnarBatch
+   * @param startRow
+   * @param size
+   * @param vectorOffset
+   */
+  public int markFilteredRows(CarbonColumnarBatch columnarBatch, int startRow, int size,
+      int vectorOffset) {
+    int rowsFiltered = 0;
+    if (currentDeleteDeltaVo != null) {
+      int len = startRow + size;
+      for (int i = startRow; i < len; i++) {
+        int rowId = pageFilteredRowId != null ? pageFilteredRowId[pageCounter][i] : i;
+        if (currentDeleteDeltaVo.containsRow(rowId)) {
+          columnarBatch.markFiltered(vectorOffset);
+          rowsFiltered++;
+        }
+        vectorOffset++;
+      }
+    }
+    return rowsFiltered;
+  }
+
+  /**
+   * Below method will be used to check row got deleted
+   *
+   * @param rowId
+   * @return is present in deleted row
+   */
+  public boolean containsDeletedRow(int rowId) {
+    if (null != currentDeleteDeltaVo) {
+      return currentDeleteDeltaVo.containsRow(rowId);
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java b/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java
new file mode 100644
index 0000000..c129161
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/RowBatch.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.core.scan.result;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.apache.carbondata.common.CarbonIterator;
+
+/**
+ * Below class holds the query result
+ */
+public class RowBatch extends CarbonIterator<Object[]> {
+
+  /**
+   * list of keys
+   */
+  protected List<Object[]> rows;
+
+  /**
+   * counter to check whether all the records are processed or not
+   */
+  protected int counter;
+
+  public RowBatch() {
+    this.rows = new ArrayList<>();
+  }
+
+  /**
+   * Below method will be used to get the rows
+   *
+   * @return
+   */
+  public List<Object[]> getRows() {
+    return rows;
+  }
+
+  /**
+   * Below method will be used to get the set the values
+   *
+   * @param rows
+   */
+  public void setRows(List<Object[]> rows) {
+    this.rows = rows;
+  }
+
+  /**
+   * This method will return one row at a time based on the counter given.
+   * @param counter
+   * @return
+   */
+  public Object[] getRawRow(int counter) {
+    return rows.get(counter);
+  }
+
+  /**
+   * For getting the total size.
+   * @return
+   */
+  public int getSize() {
+    return rows.size();
+  }
+
+
+  /**
+   * Returns {@code true} if the iteration has more elements.
+   *
+   * @return {@code true} if the iteration has more elements
+   */
+  @Override public boolean hasNext() {
+    return counter < rows.size();
+  }
+
+  /**
+   * Returns the next element in the iteration.
+   *
+   * @return the next element in the iteration
+   */
+  @Override public Object[] next() {
+    if (!hasNext()) {
+      throw new NoSuchElementException();
+    }
+    Object[] row = rows.get(counter);
+    counter++;
+    return row;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
index 8120310..bcc5634 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/FilterQueryScannedResult.java
@@ -17,7 +17,7 @@
 package org.apache.carbondata.core.scan.result.impl;
 
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
 
 /**
@@ -25,7 +25,7 @@ import org.apache.carbondata.core.scan.result.vector.ColumnVectorInfo;
  * In case of filter query data will be send
  * based on filtered row index
  */
-public class FilterQueryScannedResult extends AbstractScannedResult {
+public class FilterQueryScannedResult extends BlockletScannedResult {
 
   public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) {
     super(tableBlockExecutionInfos);
@@ -37,7 +37,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   @Override public byte[] getDictionaryKeyArray() {
     ++currentRow;
-    return getDictionaryKeyArray(rowMapping[pageCounter][currentRow]);
+    return getDictionaryKeyArray(pageFilteredRowId[pageCounter][currentRow]);
   }
 
   /**
@@ -46,7 +46,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   @Override public int[] getDictionaryKeyIntegerArray() {
     ++currentRow;
-    return getDictionaryKeyIntegerArray(rowMapping[pageCounter][currentRow]);
+    return getDictionaryKeyIntegerArray(pageFilteredRowId[pageCounter][currentRow]);
   }
 
   /**
@@ -55,7 +55,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    * @return complex type key array
    */
   @Override public byte[][] getComplexTypeKeyArray() {
-    return getComplexTypeKeyArray(rowMapping[pageCounter][currentRow]);
+    return getComplexTypeKeyArray(pageFilteredRowId[pageCounter][currentRow]);
   }
 
   /**
@@ -65,17 +65,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    * @return no dictionary key array for all the no dictionary dimension
    */
   @Override public byte[][] getNoDictionaryKeyArray() {
-    return getNoDictionaryKeyArray(rowMapping[pageCounter][currentRow]);
-  }
-
-  /**
-   * Below method will be used to get the no dictionary key
-   * string array for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  @Override public String[] getNoDictionaryKeyStringArray() {
-    return getNoDictionaryKeyStringArray(rowMapping[pageCounter][currentRow]);
+    return getNoDictionaryKeyArray(pageFilteredRowId[pageCounter][currentRow]);
   }
 
   /**
@@ -84,7 +74,7 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    * @return valid row id
    */
   @Override public int getCurrentRowId() {
-    return rowMapping[pageCounter][currentRow];
+    return pageFilteredRowId[pageCounter][currentRow];
   }
 
   /**
@@ -92,10 +82,12 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   public void fillColumnarDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
     int column = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[dictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(rowMapping[pageCounter], vectorInfo, column,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
+    for (int chunkIndex : this.dictionaryColumnChunkIndexes) {
+      column = dimensionColumnPages[chunkIndex][pageCounter].fillVector(
+          pageFilteredRowId[pageCounter],
+          vectorInfo,
+          column,
+          columnGroupKeyStructureInfo.get(chunkIndex));
     }
   }
 
@@ -104,10 +96,12 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   public void fillColumnarNoDictionaryBatch(ColumnVectorInfo[] vectorInfo) {
     int column = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      column = dimensionDataChunks[noDictionaryColumnBlockIndexes[i]][pageCounter]
-          .fillConvertedChunkData(rowMapping[pageCounter], vectorInfo, column,
-              columnGroupKeyStructureInfo.get(noDictionaryColumnBlockIndexes[i]));
+    for (int chunkIndex : this.noDictionaryColumnChunkIndexes) {
+      column = dimensionColumnPages[chunkIndex][pageCounter].fillVector(
+          pageFilteredRowId[pageCounter],
+          vectorInfo,
+          column,
+          columnGroupKeyStructureInfo.get(chunkIndex));
     }
   }
 
@@ -116,8 +110,10 @@ public class FilterQueryScannedResult extends AbstractScannedResult {
    */
   public void fillColumnarMeasureBatch(ColumnVectorInfo[] vectorInfo, int[] measuresOrdinal) {
     for (int i = 0; i < measuresOrdinal.length; i++) {
-      vectorInfo[i].measureVectorFiller.fillMeasureVectorForFilter(rowMapping[pageCounter],
-          measureDataChunks[measuresOrdinal[i]][pageCounter], vectorInfo[i]);
+      vectorInfo[i].measureVectorFiller.fillMeasureVector(
+          pageFilteredRowId[pageCounter],
+          measureColumnPages[measuresOrdinal[i]][pageCounter],
+          vectorInfo[i]);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
index 3978f9e..06687c2 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/impl/NonFilterQueryScannedResult.java
@@ -17,14 +17,14 @@
 package org.apache.carbondata.core.scan.result.impl;
 
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
-import org.apache.carbondata.core.scan.result.AbstractScannedResult;
+import org.apache.carbondata.core.scan.result.BlockletScannedResult;
 
 /**
  * Result provide class for non filter query
  * In case of no filter query we need to return
  * complete data
  */
-public class NonFilterQueryScannedResult extends AbstractScannedResult {
+public class NonFilterQueryScannedResult extends BlockletScannedResult {
 
   public NonFilterQueryScannedResult(BlockExecutionInfo blockExecutionInfo) {
     super(blockExecutionInfo);
@@ -68,16 +68,6 @@ public class NonFilterQueryScannedResult extends AbstractScannedResult {
   }
 
   /**
-   * Below method will be used to get the no dictionary key
-   * string array for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  @Override public String[] getNoDictionaryKeyStringArray() {
-    return getNoDictionaryKeyStringArray(currentRow);
-  }
-
-  /**
    * will return the current valid row id
    *
    * @return valid row id

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
index 6172b40..4e628fe 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -28,18 +28,17 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastore.DataRefNode;
 import org.apache.carbondata.core.datastore.DataRefNodeFinder;
-import org.apache.carbondata.core.datastore.FileHolder;
+import org.apache.carbondata.core.datastore.FileReader;
 import org.apache.carbondata.core.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.datastore.impl.btree.BTreeDataRefNodeFinder;
-import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNodeWrapper;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataRefNode;
 import org.apache.carbondata.core.mutate.DeleteDeltaVo;
 import org.apache.carbondata.core.reader.CarbonDeleteFilesDataReader;
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.executor.infos.DeleteDeltaInfo;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.processor.AbstractDataBlockIterator;
-import org.apache.carbondata.core.scan.processor.impl.DataBlockIteratorImpl;
+import org.apache.carbondata.core.scan.processor.DataBlockIterator;
 import org.apache.carbondata.core.scan.result.vector.CarbonColumnarBatch;
 import org.apache.carbondata.core.stats.QueryStatistic;
 import org.apache.carbondata.core.stats.QueryStatisticsConstants;
@@ -63,23 +62,23 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
   private static final Map<DeleteDeltaInfo, Object> deleteDeltaToLockObjectMap =
       new ConcurrentHashMap<>();
 
-  protected ExecutorService execService;
+  private ExecutorService execService;
   /**
    * execution info of the block
    */
-  protected List<BlockExecutionInfo> blockExecutionInfos;
+  private List<BlockExecutionInfo> blockExecutionInfos;
 
   /**
    * file reader which will be used to execute the query
    */
-  protected FileHolder fileReader;
+  protected FileReader fileReader;
 
-  protected AbstractDataBlockIterator dataBlockIterator;
+  DataBlockIterator dataBlockIterator;
 
   /**
    * QueryStatisticsRecorder
    */
-  protected QueryStatisticsRecorder recorder;
+  private QueryStatisticsRecorder recorder;
   /**
    * number of cores which can be used
    */
@@ -89,7 +88,7 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
    */
   private QueryStatisticsModel queryStatisticsModel;
 
-  public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel,
+  AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel,
       ExecutorService execService) {
     String batchSizeString =
         CarbonProperties.getInstance().getProperty(CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE);
@@ -107,7 +106,6 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     this.blockExecutionInfos = infos;
     this.fileReader = FileFactory.getFileHolder(
         FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getTablePath()));
-    this.fileReader.setQueryId(queryModel.getQueryId());
     this.fileReader.setReadPageByPage(queryModel.isReadPageByPage());
     this.execService = execService;
     intialiseInfos();
@@ -130,22 +128,21 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
         blockInfo.setDeletedRecordsMap(deletedRowsMap);
       }
       DataRefNode dataRefNode = blockInfo.getDataBlock().getDataRefNode();
-      if (dataRefNode instanceof BlockletDataRefNodeWrapper) {
-        BlockletDataRefNodeWrapper wrapper = (BlockletDataRefNodeWrapper) dataRefNode;
-        blockInfo.setFirstDataBlock(wrapper);
-        blockInfo.setNumberOfBlockToScan(wrapper.numberOfNodes());
-
+      if (dataRefNode instanceof BlockletDataRefNode) {
+        BlockletDataRefNode node = (BlockletDataRefNode) dataRefNode;
+        blockInfo.setFirstDataBlock(node);
+        blockInfo.setNumberOfBlockToScan(node.numberOfNodes());
       } else {
         DataRefNode startDataBlock =
             finder.findFirstDataBlock(dataRefNode, blockInfo.getStartKey());
-        while (startDataBlock.nodeNumber() < blockInfo.getStartBlockletIndex()) {
+        while (startDataBlock.nodeIndex() < blockInfo.getStartBlockletIndex()) {
           startDataBlock = startDataBlock.getNextDataRefNode();
         }
         long numberOfBlockToScan = blockInfo.getNumberOfBlockletToScan();
         //if number of block is less than 0 then take end block.
         if (numberOfBlockToScan <= 0) {
           DataRefNode endDataBlock = finder.findLastDataBlock(dataRefNode, blockInfo.getEndKey());
-          numberOfBlockToScan = endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
+          numberOfBlockToScan = endDataBlock.nodeIndex() - startDataBlock.nodeIndex() + 1;
         }
         blockInfo.setFirstDataBlock(startDataBlock);
         blockInfo.setNumberOfBlockToScan(numberOfBlockToScan);
@@ -230,7 +227,8 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     }
   }
 
-  @Override public boolean hasNext() {
+  @Override
+  public boolean hasNext() {
     if ((dataBlockIterator != null && dataBlockIterator.hasNext())) {
       return true;
     } else if (blockExecutionInfos.size() > 0) {
@@ -240,7 +238,7 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     }
   }
 
-  protected void updateDataBlockIterator() {
+  void updateDataBlockIterator() {
     if (dataBlockIterator == null || !dataBlockIterator.hasNext()) {
       dataBlockIterator = getDataBlockIterator();
       while (dataBlockIterator != null && !dataBlockIterator.hasNext()) {
@@ -249,17 +247,17 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     }
   }
 
-  private DataBlockIteratorImpl getDataBlockIterator() {
+  private DataBlockIterator getDataBlockIterator() {
     if (blockExecutionInfos.size() > 0) {
       BlockExecutionInfo executionInfo = blockExecutionInfos.get(0);
       blockExecutionInfos.remove(executionInfo);
-      return new DataBlockIteratorImpl(executionInfo, fileReader, batchSize, queryStatisticsModel,
+      return new DataBlockIterator(executionInfo, fileReader, batchSize, queryStatisticsModel,
           execService);
     }
     return null;
   }
 
-  protected void initQueryStatiticsModel() {
+  private void initQueryStatiticsModel() {
     this.queryStatisticsModel = new QueryStatisticsModel();
     this.queryStatisticsModel.setRecorder(recorder);
     QueryStatistic queryStatisticTotalBlocklet = new QueryStatistic();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java
index 1efac30..1235789 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/ChunkRowIterator.java
@@ -18,7 +18,7 @@
 package org.apache.carbondata.core.scan.result.iterator;
 
 import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 
 /**
  * Iterator over row result
@@ -28,14 +28,14 @@ public class ChunkRowIterator extends CarbonIterator<Object[]> {
   /**
    * iterator over chunk result
    */
-  private CarbonIterator<BatchResult> iterator;
+  private CarbonIterator<RowBatch> iterator;
 
   /**
    * currect chunk
    */
-  private BatchResult currentchunk;
+  private RowBatch currentchunk;
 
-  public ChunkRowIterator(CarbonIterator<BatchResult> iterator) {
+  public ChunkRowIterator(CarbonIterator<RowBatch> iterator) {
     this.iterator = iterator;
     if (iterator.hasNext()) {
       currentchunk = iterator.next();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/975725a4/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java
index 747f5a9..c073c78 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/DetailQueryResultIterator.java
@@ -21,14 +21,14 @@ import java.util.concurrent.ExecutorService;
 
 import org.apache.carbondata.core.scan.executor.infos.BlockExecutionInfo;
 import org.apache.carbondata.core.scan.model.QueryModel;
-import org.apache.carbondata.core.scan.result.BatchResult;
+import org.apache.carbondata.core.scan.result.RowBatch;
 
 /**
  * In case of detail query we cannot keep all the records in memory so for
  * executing that query are returning a iterator over block and every time next
  * call will come it will execute the block and return the result
  */
-public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator<BatchResult> {
+public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator<RowBatch> {
 
   private final Object lock = new Object();
 
@@ -37,18 +37,18 @@ public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator
     super(infos, queryModel, execService);
   }
 
-  @Override public BatchResult next() {
+  @Override public RowBatch next() {
     return getBatchResult();
   }
 
-  private BatchResult getBatchResult() {
-    BatchResult batchResult = new BatchResult();
+  private RowBatch getBatchResult() {
+    RowBatch rowBatch = new RowBatch();
     synchronized (lock) {
       updateDataBlockIterator();
       if (dataBlockIterator != null) {
-        batchResult.setRows(dataBlockIterator.next());
+        rowBatch.setRows(dataBlockIterator.next());
       }
     }
-    return batchResult;
+    return rowBatch;
   }
 }


[04/49] carbondata git commit: [CARBONDATA-1827] S3 Carbon Implementation

Posted by ja...@apache.org.
[CARBONDATA-1827] S3 Carbon Implementation

1.Provide support for s3 in carbondata.
2.Added S3Example to create carbon table on s3.
3.Added S3CSVExample to load carbon table using csv from s3.

This closes #1805


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

Branch: refs/heads/carbonstore-rebase4
Commit: fc93f111e13c8ed40c27c98ea07abbcc9a6b0b90
Parents: c209508
Author: SangeetaGulia <sa...@knoldus.in>
Authored: Thu Sep 21 14:56:26 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Mon Feb 26 23:55:29 2018 +0800

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  21 +++
 .../filesystem/AbstractDFSCarbonFile.java       |  20 ++-
 .../datastore/filesystem/HDFSCarbonFile.java    |   5 +-
 .../core/datastore/impl/FileFactory.java        |  11 +-
 .../core/locks/CarbonLockFactory.java           |  28 ++--
 .../carbondata/core/locks/S3FileLock.java       | 111 +++++++++++++
 .../carbondata/core/util/CarbonProperties.java  |   3 +-
 .../filesystem/HDFSCarbonFileTest.java          |   8 +-
 examples/spark2/pom.xml                         |   5 +
 examples/spark2/src/main/resources/data1.csv    |  11 ++
 .../carbondata/examples/S3CsvExample.scala      |  99 +++++++++++
 .../apache/carbondata/examples/S3Example.scala  | 164 +++++++++++++++++++
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  42 ++++-
 integration/spark2/pom.xml                      |  43 +++++
 .../spark/rdd/CarbonDataRDDFactory.scala        |   3 +-
 .../org/apache/spark/sql/CarbonSession.scala    |   3 +
 16 files changed, 554 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 6e6482d..2e169c2 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -167,6 +167,22 @@ public final class CarbonCommonConstants {
   public static final String S3N_PREFIX = "s3n://";
 
   public static final String S3A_PREFIX = "s3a://";
+  /**
+   * Access Key for s3n
+   */
+  public static final String S3N_ACCESS_KEY = "fs.s3n.awsAccessKeyId";
+  /**
+   * Secret Key for s3n
+   */
+  public static final String S3N_SECRET_KEY = "fs.s3n.awsSecretAccessKey";
+  /**
+   * Access Key for s3
+   */
+  public static final String S3_ACCESS_KEY = "fs.s3.awsAccessKeyId";
+  /**
+   * Secret Key for s3
+   */
+  public static final String S3_SECRET_KEY = "fs.s3.awsSecretAccessKey";
 
   /**
    * FS_DEFAULT_FS
@@ -941,6 +957,11 @@ public final class CarbonCommonConstants {
   public static final String CARBON_LOCK_TYPE_HDFS = "HDFSLOCK";
 
   /**
+   * S3LOCK TYPE
+   */
+  public static final String CARBON_LOCK_TYPE_S3 = "S3LOCK";
+
+  /**
    * Invalid filter member log string
    */
   public static final String FILTER_INVALID_MEMBER =

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
index 68eaa21..fd5dc40 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/AbstractDFSCarbonFile.java
@@ -51,7 +51,7 @@ import org.apache.hadoop.io.compress.GzipCodec;
 import org.apache.hadoop.io.compress.Lz4Codec;
 import org.apache.hadoop.io.compress.SnappyCodec;
 
-public abstract  class AbstractDFSCarbonFile implements CarbonFile {
+public abstract class AbstractDFSCarbonFile implements CarbonFile {
   /**
    * LOGGER
    */
@@ -262,18 +262,28 @@ public abstract  class AbstractDFSCarbonFile implements CarbonFile {
   @Override public DataOutputStream getDataOutputStream(String path, FileFactory.FileType fileType,
       int bufferSize, boolean append) throws IOException {
     Path pt = new Path(path);
-    FileSystem fs = pt.getFileSystem(FileFactory.getConfiguration());
+    FileSystem fileSystem = pt.getFileSystem(FileFactory.getConfiguration());
     FSDataOutputStream stream = null;
     if (append) {
       // append to a file only if file already exists else file not found
       // exception will be thrown by hdfs
       if (CarbonUtil.isFileExists(path)) {
-        stream = fs.append(pt, bufferSize);
+        if (FileFactory.FileType.S3 == fileType) {
+          DataInputStream dataInputStream = fileSystem.open(pt);
+          int count = dataInputStream.available();
+          // create buffer
+          byte[] byteStreamBuffer = new byte[count];
+          dataInputStream.read(byteStreamBuffer);
+          stream = fileSystem.create(pt, true, bufferSize);
+          stream.write(byteStreamBuffer);
+        } else {
+          stream = fileSystem.append(pt, bufferSize);
+        }
       } else {
-        stream = fs.create(pt, true, bufferSize);
+        stream = fileSystem.create(pt, true, bufferSize);
       }
     } else {
-      stream = fs.create(pt, true, bufferSize);
+      stream = fileSystem.create(pt, true, bufferSize);
     }
     return stream;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFile.java b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFile.java
index d470b47..892a556 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFile.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFile.java
@@ -107,8 +107,11 @@ public class HDFSCarbonFile extends AbstractDFSCarbonFile {
         ((DistributedFileSystem) fs).rename(fileStatus.getPath(), new Path(changetoName),
             org.apache.hadoop.fs.Options.Rename.OVERWRITE);
         return true;
+      } else if (fileStatus.getPath().toString().startsWith("s3n")) {
+        fs.delete(new Path(changetoName), true);
+        return fs.rename(fileStatus.getPath(), new Path(changetoName));
       } else {
-        return false;
+        return fs.rename(fileStatus.getPath(), new Path(changetoName));
       }
     } catch (IOException e) {
       LOGGER.error("Exception occured: " + e.getMessage());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
index 38ed2b7..f141991 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
@@ -246,7 +246,15 @@ public final class FileFactory {
    */
   public static DataOutputStream getDataOutputStreamUsingAppend(String path, FileType fileType)
       throws IOException {
-    return getCarbonFile(path).getDataOutputStreamUsingAppend(path, fileType);
+    if (FileType.S3 == fileType) {
+      CarbonFile carbonFile = getCarbonFile(path);
+      if (carbonFile.exists()) {
+        carbonFile.delete();
+      }
+      return carbonFile.getDataOutputStream(path,fileType);
+    } else {
+      return getCarbonFile(path).getDataOutputStreamUsingAppend(path, fileType);
+    }
   }
 
   /**
@@ -423,6 +431,7 @@ public final class FileFactory {
       throws IOException {
     FileFactory.FileType fileType = FileFactory.getFileType(directoryPath);
     switch (fileType) {
+      case S3:
       case HDFS:
       case VIEWFS:
         try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
index e70e655..3226a63 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
@@ -52,18 +52,21 @@ public class CarbonLockFactory {
    */
   public static ICarbonLock getCarbonLockObj(AbsoluteTableIdentifier absoluteTableIdentifier,
       String lockFile) {
-    switch (lockTypeConfigured) {
-      case CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL:
-        return new LocalFileLock(absoluteTableIdentifier, lockFile);
 
-      case CarbonCommonConstants.CARBON_LOCK_TYPE_ZOOKEEPER:
-        return new ZooKeeperLocking(absoluteTableIdentifier, lockFile);
-
-      case CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS:
-        return new HdfsFileLock(absoluteTableIdentifier, lockFile);
-
-      default:
-        throw new UnsupportedOperationException("Not supported the lock type");
+    String tablePath = absoluteTableIdentifier.getTablePath();
+    if (lockTypeConfigured.equals(CarbonCommonConstants.CARBON_LOCK_TYPE_ZOOKEEPER)) {
+      return new ZooKeeperLocking(absoluteTableIdentifier, lockFile);
+    } else if (tablePath.startsWith(CarbonCommonConstants.S3A_PREFIX) ||
+        tablePath.startsWith(CarbonCommonConstants.S3N_PREFIX) ||
+            tablePath.startsWith(CarbonCommonConstants.S3_PREFIX)) {
+      lockTypeConfigured = CarbonCommonConstants.CARBON_LOCK_TYPE_S3;
+      return new S3FileLock(absoluteTableIdentifier, lockFile);
+    } else if (tablePath.startsWith(CarbonCommonConstants.HDFSURL_PREFIX)) {
+      lockTypeConfigured = CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS;
+      return new HdfsFileLock(absoluteTableIdentifier, lockFile);
+    } else {
+      lockTypeConfigured = CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL;
+      return new LocalFileLock(absoluteTableIdentifier, lockFile);
     }
   }
 
@@ -84,6 +87,9 @@ public class CarbonLockFactory {
       case CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS:
         return new HdfsFileLock(locFileLocation, lockFile);
 
+      case CarbonCommonConstants.CARBON_LOCK_TYPE_S3:
+        return new S3FileLock(locFileLocation, lockFile);
+
       default:
         throw new UnsupportedOperationException("Not supported the lock type");
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java b/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java
new file mode 100644
index 0000000..8836960
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/locks/S3FileLock.java
@@ -0,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.apache.carbondata.core.locks;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+
+/**
+ * This class is used to handle the S3 File locking.
+ * This is acheived using the concept of acquiring the data out stream using Append option.
+ */
+public class S3FileLock extends AbstractCarbonLock {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(S3FileLock.class.getName());
+  /**
+   * location s3 file location
+   */
+  private String location;
+
+  private DataOutputStream dataOutputStream;
+
+  /**
+   * @param tableIdentifier
+   * @param lockFile
+   */
+  public S3FileLock(AbsoluteTableIdentifier tableIdentifier, String lockFile) {
+    this(tableIdentifier.getTablePath(), lockFile);
+  }
+
+  /**
+   * @param lockFileLocation
+   * @param lockFile
+   */
+  public S3FileLock(String lockFileLocation, String lockFile) {
+    this.location = lockFileLocation + CarbonCommonConstants.FILE_SEPARATOR + lockFile;
+    LOGGER.info("S3 lock path:" + this.location);
+    initRetry();
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.carbondata.core.locks.ICarbonLock#unlock()
+   */
+  @Override public boolean unlock() {
+    boolean status = false;
+    if (null != dataOutputStream) {
+      try {
+        dataOutputStream.close();
+        status = true;
+      } catch (IOException e) {
+        status = false;
+      } finally {
+        CarbonFile carbonFile =
+            FileFactory.getCarbonFile(location, FileFactory.getFileType(location));
+        if (carbonFile.exists()) {
+          if (carbonFile.delete()) {
+            LOGGER.info("Deleted the lock file " + location);
+          } else {
+            LOGGER.error("Not able to delete the lock file " + location);
+            status = false;
+          }
+        } else {
+          LOGGER.error(
+              "Not able to delete the lock file because it is not existed in location " + location);
+          status = false;
+        }
+      }
+    }
+    return status;
+  }
+
+  /* (non-Javadoc)
+   * @see org.apache.carbondata.core.locks.ICarbonLock#lock()
+   */
+  @Override public boolean lock() {
+    try {
+      if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
+        FileFactory.createNewLockFile(location, FileFactory.getFileType(location));
+      }
+      dataOutputStream =
+          FileFactory.getDataOutputStreamUsingAppend(location, FileFactory.getFileType(location));
+      return true;
+    } catch (IOException e) {
+      LOGGER.error(e, e.getMessage());
+      return false;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index 3dc7b8f..9d52669 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -369,7 +369,8 @@ public final class CarbonProperties {
     String defaultFs = configuration.get("fs.defaultFS");
     if (null != defaultFs && (defaultFs.startsWith(CarbonCommonConstants.HDFSURL_PREFIX)
         || defaultFs.startsWith(CarbonCommonConstants.VIEWFSURL_PREFIX) || defaultFs
-        .startsWith(CarbonCommonConstants.ALLUXIOURL_PREFIX))
+        .startsWith(CarbonCommonConstants.ALLUXIOURL_PREFIX) || defaultFs
+        .startsWith(CarbonCommonConstants.S3A_PREFIX))
         && !CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS.equalsIgnoreCase(lockTypeConfigured)) {
       LOGGER.warn("The value \"" + lockTypeConfigured + "\" configured for key "
           + LOCK_TYPE + " is invalid for current file system. "

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java b/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java
index 7726693..4018123 100644
--- a/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/datastore/filesystem/HDFSCarbonFileTest.java
@@ -369,7 +369,13 @@ public class HDFSCarbonFileTest {
             }
 
         };
-        assertEquals(hdfsCarbonFile.renameForce(fileName), false);
+        new MockUp<WebHdfsFileSystem>(){
+            @Mock
+            public boolean rename(final Path src, final Path dst) throws IOException {
+                return true;
+            }
+        };
+        assertEquals(hdfsCarbonFile.renameForce(fileName), true);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/examples/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/examples/spark2/pom.xml b/examples/spark2/pom.xml
index c17f0ee..f64dc9f 100644
--- a/examples/spark2/pom.xml
+++ b/examples/spark2/pom.xml
@@ -62,6 +62,11 @@
       <version>${spark.version}</version>
       <scope>${spark.deps.scope}</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpclient</artifactId>
+      <version>4.2</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/examples/spark2/src/main/resources/data1.csv
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/resources/data1.csv b/examples/spark2/src/main/resources/data1.csv
new file mode 100644
index 0000000..cf732eb
--- /dev/null
+++ b/examples/spark2/src/main/resources/data1.csv
@@ -0,0 +1,11 @@
+shortField,intField,bigintField,doubleField,stringField,timestampField,decimalField,dateField,charField,floatField
+1,10,1100,48.4,spark,2015-4-23 12:01:01,1.23,2015-4-23,aaa,2.5
+5,17,1140,43.4,spark,2015-7-27 12:01:02,3.45,2015-7-27,bbb,2.5
+1,11,1100,44.4,flink,2015-5-23 12:01:03,23.23,2015-5-23,ccc,2.5
+1,10,1150,43.4,spark,2015-7-24 12:01:04,254.12,2015-7-24,ddd,2.5
+1,10,1100,47.4,spark,2015-7-23 12:01:05,876.14,2015-7-23,eeee,3.5
+3,14,1160,43.4,hive,2015-7-26 12:01:06,3454.32,2015-7-26,ff,2.5
+2,10,1100,43.4,impala,2015-7-23 12:01:07,456.98,2015-7-23,ggg,2.5
+1,10,1100,43.4,spark,2015-5-23 12:01:08,32.53,2015-5-23,hhh,2.5
+4,16,1130,42.4,impala,2015-7-23 12:01:09,67.23,2015-7-23,iii,2.5
+1,10,1100,43.4,spark,2015-7-23 12:01:10,832.23,2015-7-23,jjj,2.5

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3CsvExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3CsvExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3CsvExample.scala
new file mode 100644
index 0000000..b37fba8
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3CsvExample.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.examples
+
+import java.io.File
+
+import org.apache.hadoop.fs.s3a.Constants.{ACCESS_KEY, SECRET_KEY}
+import org.apache.spark.sql.SparkSession
+import org.slf4j.{Logger, LoggerFactory}
+
+object S3CsvExample {
+
+  /**
+   * This example demonstrate to create local store and load data from CSV files on S3
+   *
+   * @param args require three parameters "Access-key" "Secret-key"
+   *             "s3 path to csv" "spark-master"
+   */
+  def main(args: Array[String]) {
+    val rootPath = new File(this.getClass.getResource("/").getPath
+                            + "../../../..").getCanonicalPath
+    val logger: Logger = LoggerFactory.getLogger(this.getClass)
+
+    import org.apache.spark.sql.CarbonSession._
+    if (args.length != 4) {
+      logger.error("Usage: java CarbonS3Example <access-key> <secret-key>" +
+                   "<s3.csv.location> <spark-master>")
+      System.exit(0)
+    }
+
+    val spark = SparkSession
+      .builder()
+      .master(args(3))
+      .appName("S3CsvExample")
+      .config("spark.driver.host", "localhost")
+      .config("spark.hadoop." + ACCESS_KEY, args(0))
+      .config("spark.hadoop." + SECRET_KEY, args(1))
+      .getOrCreateCarbonSession()
+
+    spark.sparkContext.setLogLevel("INFO")
+
+    spark.sql(
+      s"""
+         | CREATE TABLE if not exists carbon_table1(
+         | shortField SHORT,
+         | intField INT,
+         | bigintField LONG,
+         | doubleField DOUBLE,
+         | stringField STRING,
+         | timestampField TIMESTAMP,
+         | decimalField DECIMAL(18,2),
+         | dateField DATE,
+         | charField CHAR(5),
+         | floatField FLOAT
+         | )
+         | STORED BY 'carbondata'
+         | LOCATION '$rootPath/examples/spark2/target/store'
+         | TBLPROPERTIES('SORT_COLUMNS'='', 'DICTIONARY_INCLUDE'='dateField, charField')
+       """.stripMargin)
+
+    spark.sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '${ args(2) }'
+         | INTO TABLE carbon_table1
+         | OPTIONS('HEADER'='true')
+       """.stripMargin)
+
+    spark.sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '${ args(2) }'
+         | INTO TABLE carbon_table1
+         | OPTIONS('HEADER'='true')
+       """.stripMargin)
+
+    spark.sql(
+      s"""
+         | SELECT *
+         | FROM carbon_table1
+      """.stripMargin).show()
+
+    spark.sql("Drop table if exists carbon_table1")
+
+    spark.stop()
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3Example.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3Example.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3Example.scala
new file mode 100644
index 0000000..d3d0a37
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/S3Example.scala
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.examples
+
+import java.io.File
+
+import org.apache.hadoop.fs.s3a.Constants.{ACCESS_KEY, ENDPOINT, SECRET_KEY}
+import org.apache.spark.sql.{Row, SparkSession}
+import org.slf4j.{Logger, LoggerFactory}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+
+object S3Example {
+
+  /**
+   * This example demonstrate usage of
+   * 1. create carbon table with storage location on object based storage
+   * like AWS S3, Huawei OBS, etc
+   * 2. load data into carbon table, the generated file will be stored on object based storage
+   * query the table.
+   *
+   * @param args require three parameters "Access-key" "Secret-key"
+   *             "table-path on s3" "s3-endpoint" "spark-master"
+   */
+  def main(args: Array[String]) {
+    val rootPath = new File(this.getClass.getResource("/").getPath
+                            + "../../../..").getCanonicalPath
+    val path = s"$rootPath/examples/spark2/src/main/resources/data1.csv"
+    val logger: Logger = LoggerFactory.getLogger(this.getClass)
+
+    import org.apache.spark.sql.CarbonSession._
+    if (args.length < 3 || args.length > 5) {
+      logger.error("Usage: java CarbonS3Example <access-key> <secret-key>" +
+                   "<table-path-on-s3> [s3-endpoint] [spark-master]")
+      System.exit(0)
+    }
+
+    val (accessKey, secretKey, endpoint) = getKeyOnPrefix(args(2))
+    val spark = SparkSession
+      .builder()
+      .master(getSparkMaster(args))
+      .appName("S3Example")
+      .config("spark.driver.host", "localhost")
+      .config(accessKey, args(0))
+      .config(secretKey, args(1))
+      .config(endpoint, getS3EndPoint(args))
+      .getOrCreateCarbonSession()
+
+    spark.sparkContext.setLogLevel("WARN")
+
+    spark.sql("Drop table if exists carbon_table")
+
+    spark.sql(
+      s"""
+         | CREATE TABLE if not exists carbon_table(
+         | shortField SHORT,
+         | intField INT,
+         | bigintField LONG,
+         | doubleField DOUBLE,
+         | stringField STRING,
+         | timestampField TIMESTAMP,
+         | decimalField DECIMAL(18,2),
+         | dateField DATE,
+         | charField CHAR(5),
+         | floatField FLOAT
+         | )
+         | STORED BY 'carbondata'
+         | LOCATION '${ args(2) }'
+         | TBLPROPERTIES('SORT_COLUMNS'='', 'DICTIONARY_INCLUDE'='dateField, charField')
+       """.stripMargin)
+
+    spark.sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '$path'
+         | INTO TABLE carbon_table
+         | OPTIONS('HEADER'='true')
+       """.stripMargin)
+
+    spark.sql(
+      s"""
+         | SELECT *
+         | FROM carbon_table
+      """.stripMargin).show()
+
+    spark.sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '$path'
+         | INTO TABLE carbon_table
+         | OPTIONS('HEADER'='true')
+       """.stripMargin)
+
+    spark.sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '$path'
+         | INTO TABLE carbon_table
+         | OPTIONS('HEADER'='true')
+       """.stripMargin)
+
+    val countSegment: Array[Row] =
+      spark.sql(
+        s"""
+           | SHOW SEGMENTS FOR TABLE carbon_table
+       """.stripMargin).collect()
+
+    while (countSegment.length != 3) {
+      this.wait(2000)
+    }
+
+    // Use compaction command to merge segments or small files in object based storage,
+    // this can be done periodically.
+    spark.sql("ALTER table carbon_table compact 'MAJOR'")
+    spark.sql("show segments for table carbon_table").show()
+
+    spark.sql(
+      s"""
+         | SELECT *
+         | FROM carbon_table
+      """.stripMargin).show()
+
+    spark.sql("Drop table if exists carbon_table")
+
+    spark.stop()
+  }
+
+  def getKeyOnPrefix(path: String): (String, String, String) = {
+    val endPoint = "spark.hadoop." + ENDPOINT
+    if (path.startsWith(CarbonCommonConstants.S3A_PREFIX)) {
+      ("spark.hadoop." + ACCESS_KEY, "spark.hadoop." + SECRET_KEY, endPoint)
+    } else if (path.startsWith(CarbonCommonConstants.S3N_PREFIX)) {
+      ("spark.hadoop." + CarbonCommonConstants.S3N_ACCESS_KEY,
+        "spark.hadoop." + CarbonCommonConstants.S3N_SECRET_KEY, endPoint)
+    } else if (path.startsWith(CarbonCommonConstants.S3_PREFIX)) {
+      ("spark.hadoop." + CarbonCommonConstants.S3_ACCESS_KEY,
+        "spark.hadoop." + CarbonCommonConstants.S3_SECRET_KEY, endPoint)
+    } else {
+      throw new Exception("Incorrect Store Path")
+    }
+  }
+
+  def getS3EndPoint(args: Array[String]): String = {
+    if (args.length >= 4 && args(3).contains(".com")) args(3)
+    else ""
+  }
+
+  def getSparkMaster(args: Array[String]): String = {
+      if (args.length == 5) args(4)
+      else if (args(3).contains("spark:") || args(3).contains("mesos:")) args(3)
+      else "local"
+    }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 1fa1689..917fc88 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -41,6 +41,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.common.logging.impl.StandardLogService
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.compression.CompressorFactory
+import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, ThreadLocalTaskInfo}
 import org.apache.carbondata.core.util.path.CarbonTablePath
@@ -346,11 +347,31 @@ class NewDataFrameLoaderRDD[K, V](
     sc: SparkContext,
     result: DataLoadResult[K, V],
     carbonLoadModel: CarbonLoadModel,
-    prev: DataLoadCoalescedRDD[Row]) extends CarbonRDD[(K, V)](prev) {
+    prev: DataLoadCoalescedRDD[Row],
+    @transient hadoopConf: Configuration) extends CarbonRDD[(K, V)](prev) {
 
-  override def internalCompute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
+  private val confBytes = {
+    val bao = new ByteArrayOutputStream()
+    val oos = new ObjectOutputStream(bao)
+    hadoopConf.write(oos)
+    oos.close()
+    CompressorFactory.getInstance().getCompressor.compressByte(bao.toByteArray)
+  }
 
+  private def getConf = {
+    val configuration = new Configuration(false)
+    val bai = new ByteArrayInputStream(CompressorFactory.getInstance().getCompressor
+      .unCompressByte(confBytes))
+    val ois = new ObjectInputStream(bai)
+    configuration.readFields(ois)
+    ois.close()
+    configuration
+  }
+
+  override def internalCompute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+    val hadoopConf = getConf
+    setS3Configurations(hadoopConf)
     val iter = new Iterator[(K, V)] {
       val loadMetadataDetails = new LoadMetadataDetails()
       val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
@@ -420,6 +441,23 @@ class NewDataFrameLoaderRDD[K, V](
     iter
   }
   override protected def getPartitions: Array[Partition] = firstParent[Row].partitions
+
+  private def setS3Configurations(hadoopConf: Configuration): Unit = {
+    FileFactory.getConfiguration
+      .set("fs.s3a.access.key", hadoopConf.get("fs.s3a.access.key", ""))
+    FileFactory.getConfiguration
+      .set("fs.s3a.secret.key", hadoopConf.get("fs.s3a.secret.key", ""))
+    FileFactory.getConfiguration
+      .set("fs.s3a.endpoint", hadoopConf.get("fs.s3a.endpoint", ""))
+    FileFactory.getConfiguration.set(CarbonCommonConstants.S3_ACCESS_KEY,
+      hadoopConf.get(CarbonCommonConstants.S3_ACCESS_KEY, ""))
+    FileFactory.getConfiguration.set(CarbonCommonConstants.S3_SECRET_KEY,
+      hadoopConf.get(CarbonCommonConstants.S3_SECRET_KEY, ""))
+    FileFactory.getConfiguration.set(CarbonCommonConstants.S3N_ACCESS_KEY,
+      hadoopConf.get(CarbonCommonConstants.S3N_ACCESS_KEY, ""))
+    FileFactory.getConfiguration.set(CarbonCommonConstants.S3N_SECRET_KEY,
+     hadoopConf.get(CarbonCommonConstants.S3N_SECRET_KEY, ""))
+  }
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index b68a55d..aac1ff6 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -57,6 +57,49 @@
       <version>2.2.1</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-aws</artifactId>
+      <version>${hadoop.version}</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-databind</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>aws-java-sdk</artifactId>
+      <version>1.7.4</version>
+      <exclusions>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-core</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-annotations</artifactId>
+        </exclusion>
+        <exclusion>
+          <groupId>com.fasterxml.jackson.core</groupId>
+          <artifactId>jackson-databind</artifactId>
+        </exclusion>
+      </exclusions>
+    </dependency>
+    <dependency>
+      <groupId>net.java.dev.jets3t</groupId>
+      <artifactId>jets3t</artifactId>
+      <version>0.9.0</version>
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 09484c4..6f08154 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -976,7 +976,8 @@ object CarbonDataRDDFactory {
         sqlContext.sparkContext,
         new DataLoadResultImpl(),
         carbonLoadModel,
-        newRdd
+        newRdd,
+        sqlContext.sparkContext.hadoopConfiguration
       ).collect()
     } catch {
       case ex: Exception =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/fc93f111/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
index 0116d9e..935b0a6 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala
@@ -21,6 +21,7 @@ import java.io.File
 import scala.collection.JavaConverters._
 
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.s3a.Constants.{ACCESS_KEY, ENDPOINT, SECRET_KEY}
 import org.apache.spark.{SparkConf, SparkContext}
 import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
 import org.apache.spark.sql.SparkSession.Builder
@@ -30,6 +31,7 @@ import org.apache.spark.sql.internal.{SessionState, SharedState}
 import org.apache.spark.util.{CarbonReflectionUtils, Utils}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, ThreadLocalSessionInfo}
 
 /**
@@ -152,6 +154,7 @@ object CarbonSession {
             sparkConf.setAppName(randomAppName)
           }
           val sc = SparkContext.getOrCreate(sparkConf)
+          setS3Configurations(sc)
           // maybe this is an existing SparkContext, update its SparkConf which maybe used
           // by SparkSession
           options.foreach { case (k, v) => sc.conf.set(k, v) }