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 2017/10/10 03:07:48 UTC

[01/50] [abbrv] carbondata git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/carbondata [Forced Update!]

Repository: carbondata
Updated Branches:
  refs/heads/streaming_ingest 3e026c431 -> 08a82b59b (forced update)


Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/carbondata


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

Branch: refs/heads/streaming_ingest
Commit: 6c73bee0640a716082aa23c0ad2b5de0cdf30451
Parents: fc50f5b 36ceb59
Author: chenliang613 <ch...@apache.org>
Authored: Tue Sep 19 07:48:16 2017 +0800
Committer: chenliang613 <ch...@apache.org>
Committed: Tue Sep 19 07:48:16 2017 +0800

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   6 +
 .../carbondata/core/datastore/TableSpec.java    |   4 +
 ...feVariableLengthDimensionDataChunkStore.java |  11 +-
 ...afeVariableLengthDimesionDataChunkStore.java |   7 +-
 .../core/metadata/schema/PartitionInfo.java     |   5 +
 .../DictionaryBasedVectorResultCollector.java   |   3 +
 .../RestructureBasedRawResultCollector.java     |   5 +-
 .../RestructureBasedVectorResultCollector.java  |  27 +-
 .../executor/impl/AbstractQueryExecutor.java    |  18 +-
 .../scan/executor/util/RestructureUtil.java     |  21 +-
 .../scan/filter/FilterExpressionProcessor.java  |  24 +-
 .../carbondata/core/scan/filter/FilterUtil.java |  13 +-
 .../executer/RangeValueFilterExecuterImpl.java  |  10 +-
 .../executer/RestructureEvaluatorImpl.java      |   5 +-
 .../executer/RowLevelFilterExecuterImpl.java    |   2 +
 ...velRangeLessThanEqualFilterExecuterImpl.java |   2 +
 .../RowLevelRangeLessThanFiterExecuterImpl.java |   2 +
 .../RowLevelRangeFilterResolverImpl.java        |  15 +-
 .../carbondata/core/util/DataTypeUtil.java      |  43 +-
 docs/data-management.md                         |   4 +-
 .../hadoop/api/CarbonTableInputFormat.java      |  23 +-
 integration/presto/pom.xml                      |   6 +-
 .../sdv/generated/DataLoadingTestCase.scala     |  14 +-
 .../sdv/generated/QueriesBVATestCase.scala      |   2 +-
 .../sdv/generated/QueriesBasicTestCase.scala    |   2 +-
 .../generated/QueriesCompactionTestCase.scala   |   2 +-
 .../QueriesExcludeDictionaryTestCase.scala      |   2 +-
 .../SortColumnExcudeDictTestCase.scala          | 433 +++++++++++++++++++
 .../src/test/resources/data_beyond68yrs.csv     |  11 +
 .../spark/testsuite/datetype/DateTypeTest.scala |  51 +--
 .../RangeFilterAllDataTypesTestCases.scala      |   1 +
 .../TimestampNoDictionaryColumnTestCase.scala   |  93 ++++
 .../partition/TestDDLForPartitionTable.scala    |   2 +-
 ...ForPartitionTableWithDefaultProperties.scala |   5 +-
 .../testsuite/sortcolumns/TestSortColumns.scala |  43 ++
 .../spark/partition/DropPartitionCallable.java  |  39 ++
 .../org/apache/carbondata/spark/KeyVal.scala    |   4 +-
 .../spark/rdd/AlterTableLoadPartitionRDD.scala  | 141 ++++++
 .../spark/rdd/AlterTableSplitPartitionRDD.scala | 146 -------
 .../spark/rdd/CarbonScanPartitionRDD.scala      |  29 +-
 .../apache/carbondata/spark/rdd/Compactor.scala |   3 +-
 .../spark/rdd/DataManagementFunc.scala          |  50 ++-
 .../carbondata/spark/rdd/PartitionDropper.scala | 122 ++++++
 .../spark/rdd/PartitionSplitter.scala           |  36 +-
 .../carbondata/spark/util/CommonUtil.scala      |   2 +-
 .../spark/util/GlobalDictionaryUtil.scala       |   3 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   9 +-
 .../command/carbonTableSchemaCommon.scala       |  35 +-
 .../org/apache/spark/util/PartitionUtils.scala  |  15 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   8 +-
 .../execution/command/carbonTableSchema.scala   |   5 +-
 .../createtable/TestCreateTableSyntax.scala     |  15 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        | 103 ++++-
 .../execution/command/carbonTableSchema.scala   | 145 ++++++-
 .../sql/parser/CarbonSpark2SqlParser.scala      |  16 +-
 .../partition/TestAlterPartitionTable.scala     | 177 ++++++--
 .../AlterTableValidationTestCase.scala          |  26 +-
 .../processing/merger/CarbonDataMergerUtil.java |   5 +-
 .../impl/NonDictionaryFieldConverterImpl.java   |  31 +-
 .../processing/spliter/RowResultProcessor.java  | 105 +++++
 .../spliter/RowResultSpliterProcessor.java      | 105 -----
 .../exception/AlterPartitionSliceException.java |  78 ++++
 .../exception/SliceSpliterException.java        |  78 ----
 63 files changed, 1840 insertions(+), 608 deletions(-)
----------------------------------------------------------------------



[29/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeInmemoryHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeInmemoryHolder.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeInmemoryHolder.java
deleted file mode 100644
index 20d9894..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeInmemoryHolder.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort.unsafe.holder;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator;
-
-public class UnsafeInmemoryHolder implements SortTempChunkHolder {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeInmemoryHolder.class.getName());
-
-  private int counter;
-
-  private int actualSize;
-
-  private UnsafeCarbonRowPage rowPage;
-
-  private Object[] currentRow;
-
-  private long address;
-
-  private NewRowComparator comparator;
-
-  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 NewRowComparator(rowPage.getNoDictionarySortColumnMapping());
-    this.columnSize = columnSize;
-  }
-
-  public boolean hasNext() {
-    if (counter < actualSize) {
-      return true;
-    }
-    return false;
-  }
-
-  public void readRow() {
-    currentRow = new Object[columnSize];
-    address = rowPage.getBuffer().get(counter);
-    rowPage.getRow(address + rowPage.getDataBlock().getBaseOffset(), currentRow);
-    counter++;
-  }
-
-  public Object[] getRow() {
-    return currentRow;
-  }
-
-  @Override public int compareTo(SortTempChunkHolder o) {
-    return comparator.compare(currentRow, o.getRow());
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-
-    if (!(obj instanceof UnsafeInmemoryHolder)) {
-      return false;
-    }
-
-    UnsafeInmemoryHolder o = (UnsafeInmemoryHolder)obj;
-
-    return this == o;
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode();
-  }
-
-  public int numberOfRows() {
-    return actualSize;
-  }
-
-  public void close() {
-    rowPage.freeMemory();
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeInmemoryMergeHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeInmemoryMergeHolder.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeInmemoryMergeHolder.java
deleted file mode 100644
index fa4534f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeInmemoryMergeHolder.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.sort.unsafe.holder;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.comparator.UnsafeRowComparator;
-
-/**
- * It is used for merging unsafe inmemory intermediate data
- */
-public class UnsafeInmemoryMergeHolder implements Comparable<UnsafeInmemoryMergeHolder> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeInmemoryMergeHolder.class.getName());
-
-  private int counter;
-
-  private int actualSize;
-
-  private UnsafeCarbonRowPage rowPage;
-
-  private UnsafeCarbonRowForMerge currentRow;
-
-  private long address;
-
-  private UnsafeRowComparator comparator;
-
-  private Object baseObject;
-
-  private byte index;
-
-  public UnsafeInmemoryMergeHolder(UnsafeCarbonRowPage rowPage, byte index) {
-    this.actualSize = rowPage.getBuffer().getActualSize();
-    this.rowPage = rowPage;
-    LOGGER.audit("Processing unsafe inmemory rows page with size : " + actualSize);
-    this.comparator = new UnsafeRowComparator(rowPage);
-    this.baseObject = rowPage.getDataBlock().getBaseObject();
-    currentRow = new UnsafeCarbonRowForMerge();
-    this.index = index;
-  }
-
-  public boolean hasNext() {
-    if (counter < actualSize) {
-      return true;
-    }
-    return false;
-  }
-
-  public void readRow() {
-    address = rowPage.getBuffer().get(counter);
-    currentRow = new UnsafeCarbonRowForMerge();
-    currentRow.address = address + rowPage.getDataBlock().getBaseOffset();
-    currentRow.index = index;
-    counter++;
-  }
-
-  public UnsafeCarbonRowForMerge getRow() {
-    return currentRow;
-  }
-
-  @Override public int compareTo(UnsafeInmemoryMergeHolder o) {
-    return comparator.compare(currentRow, baseObject, o.getRow(), o.getBaseObject());
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-
-    if (!(obj instanceof UnsafeInmemoryMergeHolder)) {
-      return false;
-    }
-
-    UnsafeInmemoryMergeHolder o = (UnsafeInmemoryMergeHolder)obj;
-    return this == o;
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode();
-  }
-
-  public Object getBaseObject() {
-    return baseObject;
-  }
-
-  public void close() {
-    rowPage.freeMemory();
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
deleted file mode 100644
index f5316e6..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
+++ /dev/null
@@ -1,472 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort.unsafe.holder;
-
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-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;
-import java.util.concurrent.Future;
-
-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.metadata.datatype.DataType;
-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.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-
-public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeSortTempFileChunkHolder.class.getName());
-
-  /**
-   * temp file
-   */
-  private File tempFile;
-
-  /**
-   * read stream
-   */
-  private DataInputStream stream;
-
-  /**
-   * entry count
-   */
-  private int entryCount;
-
-  /**
-   * return row
-   */
-  private Object[] returnRow;
-
-  /**
-   * number of measures
-   */
-  private int measureCount;
-
-  /**
-   * number of dimensionCount
-   */
-  private int dimensionCount;
-
-  /**
-   * number of complexDimensionCount
-   */
-  private int complexDimensionCount;
-
-  /**
-   * fileBufferSize for file reader stream size
-   */
-  private int fileBufferSize;
-
-  private Object[][] currentBuffer;
-
-  private Object[][] backupBuffer;
-
-  private boolean isBackupFilled;
-
-  private boolean prefetch;
-
-  private int bufferSize;
-
-  private int bufferRowCounter;
-
-  private ExecutorService executorService;
-
-  private Future<Void> submit;
-
-  private int prefetchRecordsProceesed;
-
-  /**
-   * sortTempFileNoOFRecordsInCompression
-   */
-  private int sortTempFileNoOFRecordsInCompression;
-
-  /**
-   * isSortTempFileCompressionEnabled
-   */
-  private boolean isSortTempFileCompressionEnabled;
-
-  /**
-   * totalRecordFetch
-   */
-  private int totalRecordFetch;
-
-  private int noDictionaryCount;
-
-  private DataType[] measureDataType;
-
-  private int numberOfObjectRead;
-  /**
-   * to store whether dimension is of dictionary type or not
-   */
-  private boolean[] isNoDictionaryDimensionColumn;
-
-  private int nullSetWordsLength;
-
-  private Comparator<Object[]> comparator;
-
-  /**
-   * Constructor to initialize
-   */
-  public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters parameters) {
-    // set temp file
-    this.tempFile = tempFile;
-
-    // set measure and dimension count
-    this.measureCount = parameters.getMeasureColCount();
-    this.dimensionCount = parameters.getDimColCount();
-    this.complexDimensionCount = parameters.getComplexDimColCount();
-
-    this.noDictionaryCount = parameters.getNoDictionaryCount();
-    // set mdkey length
-    this.fileBufferSize = parameters.getFileBufferSize();
-    this.executorService = Executors.newFixedThreadPool(1);
-    this.measureDataType = parameters.getMeasureDataType();
-    this.isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
-    this.nullSetWordsLength = ((measureCount - 1) >> 6) + 1;
-    comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
-    initialize();
-  }
-
-  /**
-   * This method will be used to initialize
-   *
-   * @throws CarbonSortKeyAndGroupByException problem while initializing
-   */
-  public void initialize() {
-    prefetch = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_PREFETCH,
-            CarbonCommonConstants.CARBON_MERGE_SORT_PREFETCH_DEFAULT));
-    bufferSize = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
-            CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT));
-    this.isSortTempFileCompressionEnabled = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED,
-            CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED_DEFAULTVALUE));
-    if (this.isSortTempFileCompressionEnabled) {
-      LOGGER.info("Compression was used while writing the sortTempFile");
-    }
-
-    try {
-      this.sortTempFileNoOFRecordsInCompression = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION,
-              CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE));
-      if (this.sortTempFileNoOFRecordsInCompression < 1) {
-        LOGGER.error("Invalid value for: "
-            + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
-            + ": Only Positive Integer value(greater than zero) is allowed.Default value will"
-            + " be used");
-
-        this.sortTempFileNoOFRecordsInCompression = Integer.parseInt(
-            CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.error(
-          "Invalid value for: " + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
-              + ", only Positive Integer value is allowed.Default value will be used");
-      this.sortTempFileNoOFRecordsInCompression = Integer
-          .parseInt(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
-    }
-
-    initialise();
-  }
-
-  private void initialise() {
-    try {
-      if (isSortTempFileCompressionEnabled) {
-        this.bufferSize = sortTempFileNoOFRecordsInCompression;
-      }
-      stream = new DataInputStream(
-          new BufferedInputStream(new FileInputStream(tempFile), this.fileBufferSize));
-      this.entryCount = stream.readInt();
-      LOGGER.audit("Processing unsafe mode file rows with size : " + entryCount);
-      if (prefetch) {
-        new DataFetcher(false).call();
-        totalRecordFetch += currentBuffer.length;
-        if (totalRecordFetch < this.entryCount) {
-          submit = executorService.submit(new DataFetcher(true));
-        }
-      } else {
-        if (isSortTempFileCompressionEnabled) {
-          new DataFetcher(false).call();
-        }
-      }
-
-    } catch (FileNotFoundException e) {
-      LOGGER.error(e);
-      throw new RuntimeException(tempFile + " No Found", e);
-    } catch (IOException e) {
-      LOGGER.error(e);
-      throw new RuntimeException(tempFile + " No Found", e);
-    } catch (Exception e) {
-      LOGGER.error(e);
-      throw new RuntimeException(tempFile + " Problem while reading", e);
-    }
-  }
-
-  /**
-   * This method will be used to read new row from file
-   *
-   * @throws CarbonSortKeyAndGroupByException problem while reading
-   */
-  public void readRow() throws CarbonSortKeyAndGroupByException {
-    if (prefetch) {
-      fillDataForPrefetch();
-    } else if (isSortTempFileCompressionEnabled) {
-      if (bufferRowCounter >= bufferSize) {
-        try {
-          new DataFetcher(false).call();
-          bufferRowCounter = 0;
-        } catch (Exception e) {
-          LOGGER.error(e);
-          throw new CarbonSortKeyAndGroupByException(tempFile + " Problem while reading", e);
-        }
-
-      }
-      prefetchRecordsProceesed++;
-      returnRow = currentBuffer[bufferRowCounter++];
-    } else {
-      this.returnRow = getRowFromStream();
-    }
-  }
-
-  private void fillDataForPrefetch() {
-    if (bufferRowCounter >= bufferSize) {
-      if (isBackupFilled) {
-        bufferRowCounter = 0;
-        currentBuffer = backupBuffer;
-        totalRecordFetch += currentBuffer.length;
-        isBackupFilled = false;
-        if (totalRecordFetch < this.entryCount) {
-          submit = executorService.submit(new DataFetcher(true));
-        }
-      } else {
-        try {
-          submit.get();
-        } catch (Exception e) {
-          LOGGER.error(e);
-        }
-        bufferRowCounter = 0;
-        currentBuffer = backupBuffer;
-        isBackupFilled = false;
-        totalRecordFetch += currentBuffer.length;
-        if (totalRecordFetch < this.entryCount) {
-          submit = executorService.submit(new DataFetcher(true));
-        }
-      }
-    }
-    prefetchRecordsProceesed++;
-    returnRow = currentBuffer[bufferRowCounter++];
-  }
-
-  /**
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
-    Object[] row = new Object[dimensionCount + measureCount];
-    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 < dimensionCount; 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 < measureCount; mesCount++) {
-        if (UnsafeCarbonRowPage.isSet(words, mesCount)) {
-          switch (measureDataType[mesCount]) {
-            case SHORT:
-              row[dimensionCount + mesCount] = stream.readShort();
-              break;
-            case INT:
-              row[dimensionCount + mesCount] = stream.readInt();
-              break;
-            case LONG:
-              row[dimensionCount + mesCount] = stream.readLong();
-              break;
-            case DOUBLE:
-              row[dimensionCount + mesCount] = stream.readDouble();
-              break;
-            case DECIMAL:
-              short aShort = stream.readShort();
-              byte[] bigDecimalInBytes = new byte[aShort];
-              stream.readFully(bigDecimalInBytes);
-              row[dimensionCount + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
-              break;
-            default:
-              throw new IllegalArgumentException("unsupported data type:" +
-                  measureDataType[mesCount]);
-          }
-        }
-      }
-      return row;
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException(e);
-    }
-  }
-
-  /**
-   * below method will be used to get the row
-   *
-   * @return row
-   */
-  public Object[] getRow() {
-    return this.returnRow;
-  }
-
-  /**
-   * below method will be used to check whether any more records are present
-   * in file or not
-   *
-   * @return more row present in file
-   */
-  public boolean hasNext() {
-    if (prefetch || isSortTempFileCompressionEnabled) {
-      return this.prefetchRecordsProceesed < this.entryCount;
-    }
-    return this.numberOfObjectRead < this.entryCount;
-  }
-
-  /**
-   * Below method will be used to close streams
-   */
-  public void close() {
-    CarbonUtil.closeStreams(stream);
-    executorService.shutdown();
-  }
-
-  /**
-   * This method will number of entries
-   *
-   * @return entryCount
-   */
-  public int numberOfRows() {
-    return entryCount;
-  }
-
-  @Override public int compareTo(SortTempChunkHolder other) {
-    return comparator.compare(returnRow, other.getRow());
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-
-    if (!(obj instanceof UnsafeSortTempFileChunkHolder)) {
-      return false;
-    }
-    UnsafeSortTempFileChunkHolder o = (UnsafeSortTempFileChunkHolder) obj;
-
-    return this == o;
-  }
-
-  @Override public int hashCode() {
-    int hash = 0;
-    hash += 31 * measureCount;
-    hash += 31 * dimensionCount;
-    hash += 31 * complexDimensionCount;
-    hash += 31 * noDictionaryCount;
-    hash += tempFile.hashCode();
-    return hash;
-  }
-
-  private final class DataFetcher implements Callable<Void> {
-    private boolean isBackUpFilling;
-
-    private int numberOfRecords;
-
-    private DataFetcher(boolean backUp) {
-      isBackUpFilling = backUp;
-      calculateNumberOfRecordsToBeFetched();
-    }
-
-    private void calculateNumberOfRecordsToBeFetched() {
-      int numberOfRecordsLeftToBeRead = entryCount - totalRecordFetch;
-      numberOfRecords =
-          bufferSize < numberOfRecordsLeftToBeRead ? bufferSize : numberOfRecordsLeftToBeRead;
-    }
-
-    @Override public Void call() throws Exception {
-      try {
-        if (isBackUpFilling) {
-          backupBuffer = prefetchRecordsFromFile(numberOfRecords);
-          isBackupFilled = true;
-        } else {
-          currentBuffer = prefetchRecordsFromFile(numberOfRecords);
-        }
-      } catch (Exception e) {
-        LOGGER.error(e);
-      }
-      return null;
-    }
-
-  }
-
-  /**
-   * This method will read the records from sort temp file and keep it in a buffer
-   *
-   * @param numberOfRecords
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeInMemoryIntermediateDataMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeInMemoryIntermediateDataMerger.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeInMemoryIntermediateDataMerger.java
deleted file mode 100644
index 5480838..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeInMemoryIntermediateDataMerger.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.sort.unsafe.merger;
-
-import java.util.AbstractQueue;
-import java.util.PriorityQueue;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeCarbonRowForMerge;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeInmemoryMergeHolder;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-public class UnsafeInMemoryIntermediateDataMerger implements Runnable {
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeInMemoryIntermediateDataMerger.class.getName());
-
-  /**
-   * recordHolderHeap
-   */
-  private AbstractQueue<UnsafeInmemoryMergeHolder> recordHolderHeap;
-
-  /**
-   * fileCounter
-   */
-  private int holderCounter;
-
-  /**
-   * entryCount
-   */
-  private int entryCount;
-
-  private UnsafeCarbonRowPage[] unsafeCarbonRowPages;
-
-  private long[] mergedAddresses;
-
-  private byte[] rowPageIndexes;
-
-  /**
-   * IntermediateFileMerger Constructor
-   */
-  public UnsafeInMemoryIntermediateDataMerger(UnsafeCarbonRowPage[] unsafeCarbonRowPages,
-      int totalSize) {
-    this.holderCounter = unsafeCarbonRowPages.length;
-    this.unsafeCarbonRowPages = unsafeCarbonRowPages;
-    this.mergedAddresses = new long[totalSize];
-    this.rowPageIndexes = new byte[totalSize];
-    this.entryCount = 0;
-  }
-
-  @Override
-  public void run() {
-    long intermediateMergeStartTime = System.currentTimeMillis();
-    int holderCounterConst = holderCounter;
-    try {
-      startSorting();
-      while (hasNext()) {
-        writeDataToMemory(next());
-      }
-      double intermediateMergeCostTime =
-          (System.currentTimeMillis() - intermediateMergeStartTime) / 1000.0;
-      LOGGER.info("============================== Intermediate Merge of " + holderCounterConst
-          + " in-memory sort Cost Time: " + intermediateMergeCostTime + "(s)");
-    } catch (Exception e) {
-      LOGGER.error(e, "Problem while intermediate merging");
-    }
-  }
-
-  /**
-   * This method will be used to get the sorted record from file
-   *
-   * @return sorted record sorted record
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private UnsafeCarbonRowForMerge getSortedRecordFromMemory()
-      throws CarbonSortKeyAndGroupByException {
-    UnsafeCarbonRowForMerge row = null;
-
-    // poll the top object from heap
-    // heap maintains binary tree which is based on heap condition that will
-    // be based on comparator we are passing the heap
-    // when will call poll it will always delete root of the tree and then
-    // it does trickel down operation complexity is log(n)
-    UnsafeInmemoryMergeHolder poll = this.recordHolderHeap.poll();
-
-    // get the row from chunk
-    row = poll.getRow();
-
-    // check if there no entry present
-    if (!poll.hasNext()) {
-      // change the file counter
-      --this.holderCounter;
-
-      // reaturn row
-      return row;
-    }
-
-    // read new row
-    poll.readRow();
-
-    // add to heap
-    this.recordHolderHeap.add(poll);
-
-    // return row
-    return row;
-  }
-
-  /**
-   * Below method will be used to start storing process This method will get
-   * all the temp files present in sort temp folder then it will create the
-   * record holder heap and then it will read first record from each file and
-   * initialize the heap
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private void startSorting() throws CarbonSortKeyAndGroupByException {
-    LOGGER.info("Number of row pages in intermediate merger: " + this.holderCounter);
-
-    // create record holder heap
-    createRecordHolderQueue(unsafeCarbonRowPages);
-
-    // iterate over file list and create chunk holder and add to heap
-    LOGGER.info("Started adding first record from row page");
-
-    UnsafeInmemoryMergeHolder unsafePageHolder = null;
-    byte index = 0;
-    for (UnsafeCarbonRowPage unsafeCarbonRowPage : unsafeCarbonRowPages) {
-      // create chunk holder
-      unsafePageHolder = new UnsafeInmemoryMergeHolder(unsafeCarbonRowPage, index++);
-
-      // initialize
-      unsafePageHolder.readRow();
-
-      // add to heap
-      this.recordHolderHeap.add(unsafePageHolder);
-    }
-
-    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
-  }
-
-  /**
-   * This method will be used to create the heap which will be used to hold
-   * the chunk of data
-   */
-  private void createRecordHolderQueue(UnsafeCarbonRowPage[] pages) {
-    // creating record holder heap
-    this.recordHolderHeap = new PriorityQueue<UnsafeInmemoryMergeHolder>(pages.length);
-  }
-
-  /**
-   * This method will be used to get the sorted row
-   *
-   * @return sorted row
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private UnsafeCarbonRowForMerge next() throws CarbonSortKeyAndGroupByException {
-    return getSortedRecordFromMemory();
-  }
-
-  /**
-   * This method will be used to check whether any more element is present or
-   * not
-   *
-   * @return more element is present
-   */
-  private boolean hasNext() {
-    return this.holderCounter > 0;
-  }
-
-  /**
-   * Below method will be used to write data to file
-   */
-  private void writeDataToMemory(UnsafeCarbonRowForMerge row) {
-    mergedAddresses[entryCount] = row.address;
-    rowPageIndexes[entryCount] = row.index;
-    entryCount++;
-  }
-
-  public int getEntryCount() {
-    return entryCount;
-  }
-
-  public UnsafeCarbonRowPage[] getUnsafeCarbonRowPages() {
-    return unsafeCarbonRowPages;
-  }
-
-  public long[] getMergedAddresses() {
-    return mergedAddresses;
-  }
-
-  public byte[] getRowPageIndexes() {
-    return rowPageIndexes;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeIntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
deleted file mode 100644
index 63f6aab..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
+++ /dev/null
@@ -1,367 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.sort.unsafe.merger;
-
-import java.io.BufferedOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.AbstractQueue;
-import java.util.Arrays;
-import java.util.PriorityQueue;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.SortTempChunkHolder;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeSortTempFileChunkHolder;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.TempSortFileWriter;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.TempSortFileWriterFactory;
-
-public class UnsafeIntermediateFileMerger implements Runnable {
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeIntermediateFileMerger.class.getName());
-
-  /**
-   * recordHolderHeap
-   */
-  private AbstractQueue<SortTempChunkHolder> recordHolderHeap;
-
-  /**
-   * fileCounter
-   */
-  private int fileCounter;
-
-  /**
-   * stream
-   */
-  private DataOutputStream stream;
-
-  /**
-   * totalNumberOfRecords
-   */
-  private int totalNumberOfRecords;
-
-  /**
-   * writer
-   */
-  private TempSortFileWriter writer;
-
-  private SortParameters mergerParameters;
-
-  private File[] intermediateFiles;
-
-  private File outPutFile;
-
-  private boolean[] noDictionarycolumnMapping;
-
-  private long[] nullSetWords;
-
-  private ByteBuffer rowData;
-
-  /**
-   * IntermediateFileMerger Constructor
-   */
-  public UnsafeIntermediateFileMerger(SortParameters mergerParameters, File[] intermediateFiles,
-      File outPutFile) {
-    this.mergerParameters = mergerParameters;
-    this.fileCounter = intermediateFiles.length;
-    this.intermediateFiles = intermediateFiles;
-    this.outPutFile = outPutFile;
-    noDictionarycolumnMapping = mergerParameters.getNoDictionaryDimnesionColumn();
-    this.nullSetWords = new long[((mergerParameters.getMeasureColCount() - 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 run() {
-    long intermediateMergeStartTime = System.currentTimeMillis();
-    int fileConterConst = fileCounter;
-    boolean isFailed = false;
-    try {
-      startSorting();
-      initialize();
-      while (hasNext()) {
-        writeDataTofile(next());
-      }
-      double intermediateMergeCostTime =
-          (System.currentTimeMillis() - intermediateMergeStartTime) / 1000.0;
-      LOGGER.info("============================== Intermediate Merge of " + fileConterConst
-          + " Sort Temp Files Cost Time: " + intermediateMergeCostTime + "(s)");
-    } catch (Exception e) {
-      LOGGER.error(e, "Problem while intermediate merging");
-      isFailed = true;
-    } finally {
-      CarbonUtil.closeStreams(this.stream);
-      if (null != writer) {
-        writer.finish();
-      }
-      if (!isFailed) {
-        try {
-          finish();
-        } catch (CarbonSortKeyAndGroupByException e) {
-          LOGGER.error(e, "Problem while deleting the merge file");
-        }
-      } else {
-        if (outPutFile.delete()) {
-          LOGGER.error("Problem while deleting the merge file");
-        }
-      }
-    }
-  }
-
-  /**
-   * This method is responsible for initializing the out stream
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private void initialize() throws CarbonSortKeyAndGroupByException {
-    if (!mergerParameters.isSortFileCompressionEnabled() && !mergerParameters.isPrefetch()) {
-      try {
-        this.stream = new DataOutputStream(
-            new BufferedOutputStream(new FileOutputStream(outPutFile),
-                mergerParameters.getFileWriteBufferSize()));
-        this.stream.writeInt(this.totalNumberOfRecords);
-      } catch (FileNotFoundException e) {
-        throw new CarbonSortKeyAndGroupByException("Problem while getting the file", e);
-      } catch (IOException e) {
-        throw new CarbonSortKeyAndGroupByException("Problem while writing the data to file", e);
-      }
-    } else {
-      writer = TempSortFileWriterFactory.getInstance()
-          .getTempSortFileWriter(mergerParameters.isSortFileCompressionEnabled(),
-              mergerParameters.getDimColCount(), mergerParameters.getComplexDimColCount(),
-              mergerParameters.getMeasureColCount(), mergerParameters.getNoDictionaryCount(),
-              mergerParameters.getFileWriteBufferSize());
-      writer.initiaize(outPutFile, totalNumberOfRecords);
-    }
-  }
-
-  /**
-   * This method will be used to get the sorted record from file
-   *
-   * @return sorted record sorted record
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  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
-    // be based on comparator we are passing the heap
-    // when will call poll it will always delete root of the tree and then
-    // it does trickel down operation complexity is log(n)
-    SortTempChunkHolder poll = this.recordHolderHeap.poll();
-
-    // get the row from chunk
-    row = poll.getRow();
-
-    // check if there no entry present
-    if (!poll.hasNext()) {
-      // if chunk is empty then close the stream
-      poll.close();
-
-      // change the file counter
-      --this.fileCounter;
-
-      // reaturn row
-      return row;
-    }
-
-    // read new row
-    poll.readRow();
-
-    // add to heap
-    this.recordHolderHeap.add(poll);
-
-    // return row
-    return row;
-  }
-
-  /**
-   * Below method will be used to start storing process This method will get
-   * all the temp files present in sort temp folder then it will create the
-   * record holder heap and then it will read first record from each file and
-   * initialize the heap
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private void startSorting() throws CarbonSortKeyAndGroupByException {
-    LOGGER.info("Number of temp file: " + this.fileCounter);
-
-    // create record holder heap
-    createRecordHolderQueue(intermediateFiles);
-
-    // iterate over file list and create chunk holder and add to heap
-    LOGGER.info("Started adding first record from each file");
-
-    SortTempChunkHolder sortTempFileChunkHolder = null;
-
-    for (File tempFile : intermediateFiles) {
-      // create chunk holder
-      sortTempFileChunkHolder = new UnsafeSortTempFileChunkHolder(tempFile, mergerParameters);
-
-      sortTempFileChunkHolder.readRow();
-      this.totalNumberOfRecords += sortTempFileChunkHolder.numberOfRows();
-
-      // add to heap
-      this.recordHolderHeap.add(sortTempFileChunkHolder);
-    }
-
-    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
-  }
-
-  /**
-   * This method will be used to create the heap which will be used to hold
-   * the chunk of data
-   *
-   * @param listFiles list of temp files
-   */
-  private void createRecordHolderQueue(File[] listFiles) {
-    // creating record holder heap
-    this.recordHolderHeap = new PriorityQueue<SortTempChunkHolder>(listFiles.length);
-  }
-
-  /**
-   * This method will be used to get the sorted row
-   *
-   * @return sorted row
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private Object[] next() throws CarbonSortKeyAndGroupByException {
-    return getSortedRecordFromFile();
-  }
-
-  /**
-   * This method will be used to check whether any more element is present or
-   * not
-   *
-   * @return more element is present
-   */
-  private boolean hasNext() {
-    return this.fileCounter > 0;
-  }
-
-  /**
-   * Below method will be used to write data to file
-   *
-   * @throws CarbonSortKeyAndGroupByException problem while writing
-   */
-  private void writeDataTofile(Object[] row) throws CarbonSortKeyAndGroupByException, IOException {
-    int dimCount = 0;
-    int size = 0;
-    DataType[] type = mergerParameters.getMeasureDataType();
-    for (; dimCount < noDictionarycolumnMapping.length; dimCount++) {
-      if (noDictionarycolumnMapping[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 =
-        mergerParameters.getDimColCount() + mergerParameters.getComplexDimColCount();
-    int measureSize = mergerParameters.getMeasureColCount();
-    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 < measureSize; mesCount++) {
-      Object value = row[mesCount + dimensionSize];
-      if (null != value) {
-        switch (type[mesCount]) {
-          case SHORT:
-            rowData.putShort(size, (Short) value);
-            size += 2;
-            break;
-          case INT:
-            rowData.putInt(size, (Integer) value);
-            size += 4;
-            break;
-          case LONG:
-            rowData.putLong(size, (Long) value);
-            size += 8;
-            break;
-          case DOUBLE:
-            rowData.putDouble(size, (Double) value);
-            size += 8;
-            break;
-          case DECIMAL:
-            byte[] bigDecimalInBytes = (byte[]) value;
-            rowData.putShort(size, (short)bigDecimalInBytes.length);
-            size += 2;
-            for (int i = 0; i < bigDecimalInBytes.length; i++) {
-              rowData.put(size++, bigDecimalInBytes[i]);
-            }
-            break;
-        }
-        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 {
-    if (recordHolderHeap != null) {
-      int size = recordHolderHeap.size();
-      for (int i = 0; i < size; i++) {
-        recordHolderHeap.poll().close();
-      }
-    }
-    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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeIntermediateMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeIntermediateMerger.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeIntermediateMerger.java
deleted file mode 100644
index 49791e8..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeIntermediateMerger.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.sort.unsafe.merger;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-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.util.CarbonProperties;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-
-/**
- * It does mergesort intermediate files to big file.
- */
-public class UnsafeIntermediateMerger {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeIntermediateMerger.class.getName());
-
-  /**
-   * executorService
-   */
-  private ExecutorService executorService;
-  /**
-   * rowPages
-   */
-  private List<UnsafeCarbonRowPage> rowPages;
-
-  private List<UnsafeInMemoryIntermediateDataMerger> mergedPages;
-
-  private SortParameters parameters;
-
-  private final Object lockObject = new Object();
-
-  private boolean offHeap;
-
-  private List<File> procFiles;
-
-  public UnsafeIntermediateMerger(SortParameters parameters) {
-    this.parameters = parameters;
-    // processed file list
-    this.rowPages = new ArrayList<UnsafeCarbonRowPage>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    this.mergedPages = new ArrayList<>();
-    this.executorService = Executors.newFixedThreadPool(parameters.getNumberOfCores());
-    this.offHeap = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT,
-            CarbonCommonConstants.ENABLE_OFFHEAP_SORT_DEFAULT));
-    this.procFiles = new ArrayList<File>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-  }
-
-  public void addDataChunkToMerge(UnsafeCarbonRowPage rowPage) {
-    // add sort temp filename to and arrayList. When the list size reaches 20 then
-    // intermediate merging of sort temp files will be triggered
-    synchronized (lockObject) {
-      rowPages.add(rowPage);
-    }
-  }
-
-  public void addFileToMerge(File sortTempFile) {
-    // add sort temp filename to and arrayList. When the list size reaches 20 then
-    // intermediate merging of sort temp files will be triggered
-    synchronized (lockObject) {
-      procFiles.add(sortTempFile);
-    }
-  }
-
-  public void startFileMergingIfPossible() {
-    File[] fileList;
-    if (procFiles.size() >= parameters.getNumberOfIntermediateFileToBeMerged()) {
-      synchronized (lockObject) {
-        fileList = procFiles.toArray(new File[procFiles.size()]);
-        this.procFiles = new ArrayList<File>();
-      }
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("Sumitting request for intermediate merging no of files: " + fileList.length);
-      }
-      startIntermediateMerging(fileList);
-    }
-  }
-
-  /**
-   * Below method will be used to start the intermediate file merging
-   *
-   * @param intermediateFiles
-   */
-  private void startIntermediateMerging(File[] intermediateFiles) {
-    //pick a temp location randomly
-    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);
-    UnsafeIntermediateFileMerger merger =
-        new UnsafeIntermediateFileMerger(parameters, intermediateFiles, file);
-    executorService.execute(merger);
-  }
-
-  public void startInmemoryMergingIfPossible() throws CarbonSortKeyAndGroupByException {
-    UnsafeCarbonRowPage[] localRowPages;
-    if (rowPages.size() >= parameters.getNumberOfIntermediateFileToBeMerged()) {
-      int totalRows = 0;
-      synchronized (lockObject) {
-        totalRows = getTotalNumberOfRows(rowPages);
-        if (totalRows <= 0) {
-          return;
-        }
-        localRowPages = rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]);
-        this.rowPages = new ArrayList<>();
-      }
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("Sumitting request for intermediate merging of in-memory pages : "
-            + localRowPages.length);
-      }
-      startIntermediateMerging(localRowPages, totalRows);
-    }
-  }
-
-  /**
-   * Below method will be used to start the intermediate file merging
-   *
-   * @param rowPages
-   */
-  private void startIntermediateMerging(UnsafeCarbonRowPage[] rowPages, int totalRows)
-      throws CarbonSortKeyAndGroupByException {
-    UnsafeInMemoryIntermediateDataMerger merger =
-        new UnsafeInMemoryIntermediateDataMerger(rowPages, totalRows);
-    mergedPages.add(merger);
-    executorService.execute(merger);
-  }
-
-  private int getTotalNumberOfRows(List<UnsafeCarbonRowPage> unsafeCarbonRowPages) {
-    int totalSize = 0;
-    for (UnsafeCarbonRowPage unsafeCarbonRowPage : unsafeCarbonRowPages) {
-      totalSize += unsafeCarbonRowPage.getBuffer().getActualSize();
-    }
-    return totalSize;
-  }
-
-  public void finish() throws CarbonSortKeyAndGroupByException {
-    try {
-      executorService.shutdown();
-      executorService.awaitTermination(2, TimeUnit.DAYS);
-    } catch (InterruptedException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while shutdown the server ", e);
-    }
-  }
-
-  public void close() {
-    if (executorService.isShutdown()) {
-      executorService.shutdownNow();
-    }
-    rowPages.clear();
-    rowPages = null;
-  }
-
-  public List<UnsafeCarbonRowPage> getRowPages() {
-    return rowPages;
-  }
-
-  public List<UnsafeInMemoryIntermediateDataMerger> getMergedPages() {
-    return mergedPages;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
deleted file mode 100644
index e3bbdcb..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.sort.unsafe.merger;
-
-import java.io.File;
-import java.io.FileFilter;
-import java.util.AbstractQueue;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.PriorityQueue;
-
-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.newflow.sort.SortStepRowUtil;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.SortTempChunkHolder;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeFinalMergePageHolder;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeInmemoryHolder;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeSortTempFileChunkHolder;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-
-public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeSingleThreadFinalSortFilesMerger.class.getName());
-
-  /**
-   * fileCounter
-   */
-  private int fileCounter;
-
-  /**
-   * recordHolderHeap
-   */
-  private AbstractQueue<SortTempChunkHolder> recordHolderHeapLocal;
-
-  private SortParameters parameters;
-
-  /**
-   * tempFileLocation
-   */
-  private String[] tempFileLocation;
-
-  private String tableName;
-
-  private boolean isStopProcess;
-
-  public UnsafeSingleThreadFinalSortFilesMerger(SortParameters parameters,
-      String[] tempFileLocation) {
-    this.parameters = parameters;
-    this.tempFileLocation = tempFileLocation;
-    this.tableName = parameters.getTableName();
-  }
-
-  /**
-   * This method will be used to merger the merged files
-   *
-   */
-  public void startFinalMerge(UnsafeCarbonRowPage[] rowPages,
-      List<UnsafeInMemoryIntermediateDataMerger> merges) throws CarbonDataWriterException {
-    startSorting(rowPages, merges);
-  }
-
-  /**
-   * Below method will be used to start storing process This method will get
-   * all the temp files present in sort temp folder then it will create the
-   * record holder heap and then it will read first record from each file and
-   * initialize the heap
-   *
-   */
-  private void startSorting(UnsafeCarbonRowPage[] rowPages,
-      List<UnsafeInMemoryIntermediateDataMerger> merges) throws CarbonDataWriterException {
-    try {
-      List<File> filesToMergeSort = getFilesToMergeSort();
-      this.fileCounter = rowPages.length + filesToMergeSort.size() + merges.size();
-      if (fileCounter == 0) {
-        LOGGER.info("No files to merge sort");
-        return;
-      }
-      LOGGER.info("Number of row pages: " + this.fileCounter);
-
-      // create record holder heap
-      createRecordHolderQueue();
-
-      // iterate over file list and create chunk holder and add to heap
-      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());
-
-        // initialize
-        sortTempFileChunkHolder.readRow();
-
-        recordHolderHeapLocal.add(sortTempFileChunkHolder);
-      }
-
-      for (final UnsafeInMemoryIntermediateDataMerger merger : merges) {
-
-        SortTempChunkHolder sortTempFileChunkHolder =
-            new UnsafeFinalMergePageHolder(merger, parameters.getNoDictionarySortColumn(),
-                parameters.getDimColCount() + parameters.getComplexDimColCount() + parameters
-                    .getMeasureColCount());
-
-        // initialize
-        sortTempFileChunkHolder.readRow();
-
-        recordHolderHeapLocal.add(sortTempFileChunkHolder);
-      }
-
-      for (final File file : filesToMergeSort) {
-
-        SortTempChunkHolder sortTempFileChunkHolder =
-            new UnsafeSortTempFileChunkHolder(file, parameters);
-
-        // initialize
-        sortTempFileChunkHolder.readRow();
-
-        recordHolderHeapLocal.add(sortTempFileChunkHolder);
-      }
-
-      LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
-    } catch (Exception e) {
-      LOGGER.error(e);
-      throw new CarbonDataWriterException(e.getMessage());
-    }
-  }
-
-  private List<File> getFilesToMergeSort() {
-    FileFilter fileFilter = new FileFilter() {
-      public boolean accept(File pathname) {
-        return pathname.getName().startsWith(tableName);
-      }
-    };
-
-    // get all the merged files
-    List<File> files = new ArrayList<File>(tempFileLocation.length);
-    for (String tempLoc : tempFileLocation)
-    {
-      File[] subFiles = new File(tempLoc).listFiles(fileFilter);
-      if (null != subFiles && subFiles.length > 0)
-      {
-        files.addAll(Arrays.asList(subFiles));
-      }
-    }
-
-    return files;
-  }
-
-  /**
-   * This method will be used to create the heap which will be used to hold
-   * the chunk of data
-   */
-  private void createRecordHolderQueue() {
-    // creating record holder heap
-    this.recordHolderHeapLocal = new PriorityQueue<SortTempChunkHolder>(fileCounter);
-  }
-
-  /**
-   * This method will be used to get the sorted row
-   *
-   * @return sorted row
-   */
-  public Object[] next() {
-    return SortStepRowUtil.convertRow(getSortedRecordFromFile(), parameters);
-  }
-
-  /**
-   * This method will be used to get the sorted record from file
-   *
-   * @return sorted record sorted record
-   */
-  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
-    // be based on comparator we are passing the heap
-    // when will call poll it will always delete root of the tree and then
-    // it does trickel down operation complexity is log(n)
-    SortTempChunkHolder poll = this.recordHolderHeapLocal.poll();
-
-    // get the row from chunk
-    row = poll.getRow();
-
-    // check if there no entry present
-    if (!poll.hasNext()) {
-      // if chunk is empty then close the stream
-      poll.close();
-
-      // change the file counter
-      --this.fileCounter;
-
-      // reaturn row
-      return row;
-    }
-
-    // read new row
-    try {
-      poll.readRow();
-    } catch (Exception e) {
-      throw new CarbonDataWriterException(e.getMessage(), e);
-    }
-
-    // add to heap
-    this.recordHolderHeapLocal.add(poll);
-
-    // return row
-    return row;
-  }
-
-  /**
-   * This method will be used to check whether any more element is present or
-   * not
-   *
-   * @return more element is present
-   */
-  public boolean hasNext() {
-    return this.fileCounter > 0;
-  }
-
-  public void clear() {
-    if (null != recordHolderHeapLocal) {
-      for (SortTempChunkHolder pageHolder : recordHolderHeapLocal) {
-        pageHolder.close();
-      }
-      recordHolderHeapLocal = null;
-    }
-  }
-
-  public boolean isStopProcess() {
-    return isStopProcess;
-  }
-
-  public void setStopProcess(boolean stopProcess) {
-    isStopProcess = stopProcess;
-  }
-}


[19/50] [abbrv] carbondata git commit: [CARBONDATA-1531] Format module should support BOOLEAN

Posted by ja...@apache.org.
[CARBONDATA-1531] Format module should support BOOLEAN

format module support Boolean data type when I was developing Boolean function, and there are some error in CI after push request into github. So format module should support Boolean.

add Boolean data type
add BOOL_BYTE in Encoding

This closes #1392


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

Branch: refs/heads/streaming_ingest
Commit: b221c9826fce6f92a25ca2af8de50de6c67aece4
Parents: 28f78b2
Author: xubo245 <60...@qq.com>
Authored: Thu Sep 28 11:06:45 2017 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Sep 28 23:27:44 2017 +0800

----------------------------------------------------------------------
 format/src/main/thrift/schema.thrift | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/b221c982/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index de1a5aa..7869378 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -32,6 +32,7 @@ enum DataType {
 	DECIMAL = 5,
 	TIMESTAMP = 6,
 	DATE = 7,
+	BOOLEAN = 8,
 	ARRAY = 20,
 	STRUCT = 21,
 }
@@ -52,7 +53,8 @@ enum Encoding{
 	ADAPTIVE_DELTA_INTEGRAL = 8; // Identifies that a column is encoded using AdaptiveDeltaIntegralCodec
 	RLE_INTEGRAL = 9;     // Identifies that a column is encoded using RLECodec
 	DIRECT_STRING = 10;   // Stores string value and string length separately in page data
-  ADAPTIVE_FLOATING = 11; // Identifies that a column is encoded using AdaptiveFloatingCodec
+	ADAPTIVE_FLOATING = 11; // Identifies that a column is encoded using AdaptiveFloatingCodec
+	BOOL_BYTE = 12;   // Identifies that a column is encoded using BooleanPageCodec
 }
 
 enum PartitionType{


[25/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/UnCompressedTempSortFileWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/UnCompressedTempSortFileWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/UnCompressedTempSortFileWriter.java
new file mode 100644
index 0000000..40fe8d5
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/UnCompressedTempSortFileWriter.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.sort.sortdata;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+public class UnCompressedTempSortFileWriter extends AbstractTempSortFileWriter {
+
+  /**
+   * UnCompressedTempSortFileWriter
+   *
+   * @param writeBufferSize
+   * @param dimensionCount
+   * @param measureCount
+   */
+  public UnCompressedTempSortFileWriter(int dimensionCount, int complexDimensionCount,
+      int measureCount, int noDictionaryCount, int writeBufferSize) {
+    super(dimensionCount, complexDimensionCount, measureCount, noDictionaryCount, writeBufferSize);
+  }
+
+  public static void writeDataOutputStream(Object[][] records, DataOutputStream dataOutputStream,
+      int measureCount, int dimensionCount, int noDictionaryCount, int complexDimensionCount)
+      throws IOException {
+    Object[] row;
+    for (int recordIndex = 0; recordIndex < records.length; recordIndex++) {
+      row = records[recordIndex];
+      int fieldIndex = 0;
+
+      for (int counter = 0; counter < dimensionCount; counter++) {
+        dataOutputStream.writeInt((Integer) NonDictionaryUtil.getDimension(fieldIndex++, row));
+      }
+
+      //write byte[] of high card dims
+      if (noDictionaryCount > 0) {
+        dataOutputStream.write(NonDictionaryUtil.getByteArrayForNoDictionaryCols(row));
+      }
+      fieldIndex = 0;
+      for (int counter = 0; counter < complexDimensionCount; counter++) {
+        int complexByteArrayLength = ((byte[]) row[fieldIndex]).length;
+        dataOutputStream.writeInt(complexByteArrayLength);
+        dataOutputStream.write(((byte[]) row[fieldIndex++]));
+      }
+
+      for (int counter = 0; counter < measureCount; counter++) {
+        if (null != row[fieldIndex]) {
+          dataOutputStream.write((byte) 1);
+          dataOutputStream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
+        } else {
+          dataOutputStream.write((byte) 0);
+        }
+
+        fieldIndex++;
+      }
+
+    }
+  }
+
+  /**
+   * Below method will be used to write the sort temp file
+   *
+   * @param records
+   */
+  public void writeSortTempFile(Object[][] records) throws CarbonSortKeyAndGroupByException {
+    ByteArrayOutputStream blockDataArray = null;
+    DataOutputStream dataOutputStream = null;
+    int totalSize = 0;
+    int recordSize = 0;
+    try {
+      recordSize = (measureCount * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE) + (dimensionCount
+          * CarbonCommonConstants.INT_SIZE_IN_BYTE);
+      totalSize = records.length * recordSize;
+
+      blockDataArray = new ByteArrayOutputStream(totalSize);
+      dataOutputStream = new DataOutputStream(blockDataArray);
+
+      writeDataOutputStream(records, dataOutputStream, measureCount, dimensionCount,
+          noDictionaryCount, complexDimensionCount);
+      stream.writeInt(records.length);
+      byte[] byteArray = blockDataArray.toByteArray();
+      stream.writeInt(byteArray.length);
+      stream.write(byteArray);
+
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException(e);
+    } finally {
+      CarbonUtil.closeStreams(blockDataArray);
+      CarbonUtil.closeStreams(dataOutputStream);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/exception/CarbonSortKeyAndGroupByException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/exception/CarbonSortKeyAndGroupByException.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/exception/CarbonSortKeyAndGroupByException.java
deleted file mode 100644
index d4e4c35..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/exception/CarbonSortKeyAndGroupByException.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.processing.sortandgroupby.exception;
-
-import java.util.Locale;
-
-public class CarbonSortKeyAndGroupByException 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 CarbonSortKeyAndGroupByException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public CarbonSortKeyAndGroupByException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public CarbonSortKeyAndGroupByException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/AbstractTempSortFileWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/AbstractTempSortFileWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/AbstractTempSortFileWriter.java
deleted file mode 100644
index bd2ccec..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/AbstractTempSortFileWriter.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sortandgroupby.sortdata;
-
-import java.io.BufferedOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-public abstract class AbstractTempSortFileWriter implements TempSortFileWriter {
-
-  /**
-   * writeFileBufferSize
-   */
-  protected int writeBufferSize;
-
-  /**
-   * Measure count
-   */
-  protected int measureCount;
-
-  /**
-   * Measure count
-   */
-  protected int dimensionCount;
-
-  /**
-   * complexDimension count
-   */
-  protected int complexDimensionCount;
-
-  /**
-   * stream
-   */
-  protected DataOutputStream stream;
-
-  /**
-   * noDictionaryCount
-   */
-  protected int noDictionaryCount;
-
-  /**
-   * AbstractTempSortFileWriter
-   *
-   * @param writeBufferSize
-   * @param dimensionCount
-   * @param measureCount
-   */
-  public AbstractTempSortFileWriter(int dimensionCount, int complexDimensionCount, int measureCount,
-      int noDictionaryCount, int writeBufferSize) {
-    this.writeBufferSize = writeBufferSize;
-    this.dimensionCount = dimensionCount;
-    this.complexDimensionCount = complexDimensionCount;
-    this.measureCount = measureCount;
-    this.noDictionaryCount = noDictionaryCount;
-  }
-
-  /**
-   * Below method will be used to initialize the stream and write the entry count
-   */
-  @Override public void initiaize(File file, int entryCount)
-      throws CarbonSortKeyAndGroupByException {
-    try {
-      stream = new DataOutputStream(
-          new BufferedOutputStream(new FileOutputStream(file), writeBufferSize));
-      stream.writeInt(entryCount);
-    } catch (FileNotFoundException e1) {
-      throw new CarbonSortKeyAndGroupByException(e1);
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException(e);
-    }
-  }
-
-  /**
-   * Below method will be used to close the stream
-   */
-  @Override public void finish() {
-    CarbonUtil.closeStreams(stream);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/CompressedTempSortFileWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/CompressedTempSortFileWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/CompressedTempSortFileWriter.java
deleted file mode 100644
index e4c851a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/CompressedTempSortFileWriter.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sortandgroupby.sortdata;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.compression.CompressorFactory;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-public class CompressedTempSortFileWriter extends AbstractTempSortFileWriter {
-
-  /**
-   * CompressedTempSortFileWriter
-   *
-   * @param writeBufferSize
-   * @param dimensionCount
-   * @param measureCount
-   */
-  public CompressedTempSortFileWriter(int dimensionCount, int complexDimensionCount,
-      int measureCount, int noDictionaryCount, int writeBufferSize) {
-    super(dimensionCount, complexDimensionCount, measureCount, noDictionaryCount, writeBufferSize);
-  }
-
-  /**
-   * Below method will be used to write the sort temp file
-   *
-   * @param records
-   */
-  public void writeSortTempFile(Object[][] records) throws CarbonSortKeyAndGroupByException {
-    DataOutputStream dataOutputStream = null;
-    ByteArrayOutputStream blockDataArray = null;
-    int totalSize = 0;
-    int recordSize = 0;
-    try {
-      recordSize = (measureCount * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE) + (dimensionCount
-          * CarbonCommonConstants.INT_SIZE_IN_BYTE);
-      totalSize = records.length * recordSize;
-
-      blockDataArray = new ByteArrayOutputStream(totalSize);
-      dataOutputStream = new DataOutputStream(blockDataArray);
-
-      UnCompressedTempSortFileWriter
-          .writeDataOutputStream(records, dataOutputStream, measureCount, dimensionCount,
-              noDictionaryCount, complexDimensionCount);
-
-      stream.writeInt(records.length);
-      byte[] byteArray = CompressorFactory.getInstance().getCompressor()
-          .compressByte(blockDataArray.toByteArray());
-      stream.writeInt(byteArray.length);
-      stream.write(byteArray);
-
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException(e);
-    } finally {
-      CarbonUtil.closeStreams(blockDataArray);
-      CarbonUtil.closeStreams(dataOutputStream);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/IntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/IntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/IntermediateFileMerger.java
deleted file mode 100644
index 7c6a889..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/IntermediateFileMerger.java
+++ /dev/null
@@ -1,385 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.sortandgroupby.sortdata;
-
-import java.io.BufferedOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.AbstractQueue;
-import java.util.PriorityQueue;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-public class IntermediateFileMerger implements Runnable {
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(IntermediateFileMerger.class.getName());
-
-  /**
-   * recordHolderHeap
-   */
-  private AbstractQueue<SortTempFileChunkHolder> recordHolderHeap;
-
-  /**
-   * fileCounter
-   */
-  private int fileCounter;
-
-  /**
-   * stream
-   */
-  private DataOutputStream stream;
-
-  /**
-   * totalNumberOfRecords
-   */
-  private int totalNumberOfRecords;
-
-  /**
-   * records
-   */
-  private Object[][] records;
-
-  /**
-   * entryCount
-   */
-  private int entryCount;
-
-  /**
-   * writer
-   */
-  private TempSortFileWriter writer;
-
-  /**
-   * totalSize
-   */
-  private int totalSize;
-
-  private SortParameters mergerParameters;
-
-  private File[] intermediateFiles;
-
-  private File outPutFile;
-
-  private boolean[] noDictionarycolumnMapping;
-
-  /**
-   * IntermediateFileMerger Constructor
-   */
-  public IntermediateFileMerger(SortParameters mergerParameters, File[] intermediateFiles,
-      File outPutFile) {
-    this.mergerParameters = mergerParameters;
-    this.fileCounter = intermediateFiles.length;
-    this.intermediateFiles = intermediateFiles;
-    this.outPutFile = outPutFile;
-    noDictionarycolumnMapping = mergerParameters.getNoDictionaryDimnesionColumn();
-  }
-
-  @Override
-  public void run() {
-    long intermediateMergeStartTime = System.currentTimeMillis();
-    int fileConterConst = fileCounter;
-    boolean isFailed = false;
-    try {
-      startSorting();
-      initialize();
-      while (hasNext()) {
-        writeDataTofile(next());
-      }
-      if (mergerParameters.isSortFileCompressionEnabled() || mergerParameters.isPrefetch()) {
-        if (entryCount > 0) {
-          if (entryCount < totalSize) {
-            Object[][] temp = new Object[entryCount][];
-            System.arraycopy(records, 0, temp, 0, entryCount);
-            records = temp;
-            this.writer.writeSortTempFile(temp);
-          } else {
-            this.writer.writeSortTempFile(records);
-          }
-        }
-      }
-      double intermediateMergeCostTime =
-          (System.currentTimeMillis() - intermediateMergeStartTime) / 1000.0;
-      LOGGER.info("============================== Intermediate Merge of " + fileConterConst +
-          " Sort Temp Files Cost Time: " + intermediateMergeCostTime + "(s)");
-    } catch (Exception e) {
-      LOGGER.error(e, "Problem while intermediate merging");
-      isFailed = true;
-    } finally {
-      records = null;
-      CarbonUtil.closeStreams(this.stream);
-      if (null != writer) {
-        writer.finish();
-      }
-      if (!isFailed) {
-        try {
-          finish();
-        } catch (CarbonSortKeyAndGroupByException e) {
-          LOGGER.error(e, "Problem while deleting the merge file");
-        }
-      } else {
-        if (outPutFile.delete()) {
-          LOGGER.error("Problem while deleting the merge file");
-        }
-      }
-    }
-  }
-
-  /**
-   * This method is responsible for initializing the out stream
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private void initialize() throws CarbonSortKeyAndGroupByException {
-    if (!mergerParameters.isSortFileCompressionEnabled() && !mergerParameters.isPrefetch()) {
-      try {
-        this.stream = new DataOutputStream(
-            new BufferedOutputStream(new FileOutputStream(outPutFile),
-                mergerParameters.getFileWriteBufferSize()));
-        this.stream.writeInt(this.totalNumberOfRecords);
-      } catch (FileNotFoundException e) {
-        throw new CarbonSortKeyAndGroupByException("Problem while getting the file", e);
-      } catch (IOException e) {
-        throw new CarbonSortKeyAndGroupByException("Problem while writing the data to file", e);
-      }
-    } else {
-      writer = TempSortFileWriterFactory.getInstance()
-          .getTempSortFileWriter(mergerParameters.isSortFileCompressionEnabled(),
-              mergerParameters.getDimColCount(), mergerParameters.getComplexDimColCount(),
-              mergerParameters.getMeasureColCount(), mergerParameters.getNoDictionaryCount(),
-              mergerParameters.getFileWriteBufferSize());
-      writer.initiaize(outPutFile, totalNumberOfRecords);
-
-      if (mergerParameters.isPrefetch()) {
-        totalSize = mergerParameters.getBufferSize();
-      } else {
-        totalSize = mergerParameters.getSortTempFileNoOFRecordsInCompression();
-      }
-    }
-  }
-
-  /**
-   * This method will be used to get the sorted record from file
-   *
-   * @return sorted record sorted record
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  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
-    // be based on comparator we are passing the heap
-    // when will call poll it will always delete root of the tree and then
-    // it does trickel down operation complexity is log(n)
-    SortTempFileChunkHolder poll = this.recordHolderHeap.poll();
-
-    // get the row from chunk
-    row = poll.getRow();
-
-    // check if there no entry present
-    if (!poll.hasNext()) {
-      // if chunk is empty then close the stream
-      poll.closeStream();
-
-      // change the file counter
-      --this.fileCounter;
-
-      // reaturn row
-      return row;
-    }
-
-    // read new row
-    poll.readRow();
-
-    // add to heap
-    this.recordHolderHeap.add(poll);
-
-    // return row
-    return row;
-  }
-
-  /**
-   * Below method will be used to start storing process This method will get
-   * all the temp files present in sort temp folder then it will create the
-   * record holder heap and then it will read first record from each file and
-   * initialize the heap
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private void startSorting() throws CarbonSortKeyAndGroupByException {
-    LOGGER.info("Number of temp file: " + this.fileCounter);
-
-    // create record holder heap
-    createRecordHolderQueue(intermediateFiles);
-
-    // iterate over file list and create chunk holder and add to heap
-    LOGGER.info("Started adding first record from each file");
-
-    SortTempFileChunkHolder sortTempFileChunkHolder = null;
-
-    for (File tempFile : intermediateFiles) {
-      // create chunk holder
-      sortTempFileChunkHolder =
-          new SortTempFileChunkHolder(tempFile, mergerParameters.getDimColCount(),
-              mergerParameters.getComplexDimColCount(), mergerParameters.getMeasureColCount(),
-              mergerParameters.getFileBufferSize(), mergerParameters.getNoDictionaryCount(),
-              mergerParameters.getMeasureDataType(),
-              mergerParameters.getNoDictionaryDimnesionColumn(),
-              mergerParameters.getNoDictionarySortColumn());
-
-      // initialize
-      sortTempFileChunkHolder.initialize();
-      sortTempFileChunkHolder.readRow();
-      this.totalNumberOfRecords += sortTempFileChunkHolder.getEntryCount();
-
-      // add to heap
-      this.recordHolderHeap.add(sortTempFileChunkHolder);
-    }
-
-    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
-  }
-
-  /**
-   * This method will be used to create the heap which will be used to hold
-   * the chunk of data
-   *
-   * @param listFiles list of temp files
-   */
-  private void createRecordHolderQueue(File[] listFiles) {
-    // creating record holder heap
-    this.recordHolderHeap = new PriorityQueue<>(listFiles.length);
-  }
-
-  /**
-   * This method will be used to get the sorted row
-   *
-   * @return sorted row
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private Object[] next() throws CarbonSortKeyAndGroupByException {
-    return getSortedRecordFromFile();
-  }
-
-  /**
-   * This method will be used to check whether any more element is present or
-   * not
-   *
-   * @return more element is present
-   */
-  private boolean hasNext() {
-    return this.fileCounter > 0;
-  }
-
-  /**
-   * Below method will be used to write data to file
-   *
-   * @throws CarbonSortKeyAndGroupByException problem while writing
-   */
-  private void writeDataTofile(Object[] row) throws CarbonSortKeyAndGroupByException {
-    if (mergerParameters.isSortFileCompressionEnabled() || mergerParameters.isPrefetch()) {
-      if (entryCount == 0) {
-        records = new Object[totalSize][];
-        records[entryCount++] = row;
-        return;
-      }
-
-      records[entryCount++] = row;
-      if (entryCount == totalSize) {
-        this.writer.writeSortTempFile(records);
-        entryCount = 0;
-        records = new Object[totalSize][];
-      }
-      return;
-    }
-    try {
-      DataType[] aggType = mergerParameters.getMeasureDataType();
-      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 : noDictionarycolumnMapping) {
-        if (nodictinary) {
-          byte[] col = nonDictArray[nonDictKeyIndex++];
-          stream.writeShort(col.length);
-          stream.write(col);
-        } else {
-          stream.writeInt(mdkArray[mdkIndex++]);
-        }
-      }
-
-      int fieldIndex = 0;
-      for (int counter = 0; counter < mergerParameters.getMeasureColCount(); counter++) {
-        if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
-          stream.write((byte) 1);
-          switch (aggType[counter]) {
-            case SHORT:
-              stream.writeShort((short)NonDictionaryUtil.getMeasure(fieldIndex, row));
-              break;
-            case INT:
-              stream.writeInt((int)NonDictionaryUtil.getMeasure(fieldIndex, row));
-              break;
-            case LONG:
-              stream.writeLong((long)NonDictionaryUtil.getMeasure(fieldIndex, row));
-              break;
-            case DOUBLE:
-              stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
-              break;
-            case DECIMAL:
-              byte[] bigDecimalInBytes = (byte[]) NonDictionaryUtil.getMeasure(fieldIndex, row);
-              stream.writeInt(bigDecimalInBytes.length);
-              stream.write(bigDecimalInBytes);
-              break;
-            default:
-              throw new IllegalArgumentException("unsupported data type:" + aggType[counter]);
-          }
-        } else {
-          stream.write((byte) 0);
-        }
-        fieldIndex++;
-      }
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
-    }
-  }
-
-  private void finish() throws CarbonSortKeyAndGroupByException {
-    if (recordHolderHeap != null) {
-      int size = recordHolderHeap.size();
-      for (int i = 0; i < size; i++) {
-        recordHolderHeap.poll().closeStream();
-      }
-    }
-    try {
-      CarbonUtil.deleteFiles(intermediateFiles);
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while deleting the intermediate files");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/NewRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/NewRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/NewRowComparator.java
deleted file mode 100644
index 247251e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/NewRowComparator.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.processing.sortandgroupby.sortdata;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
-
-public class NewRowComparator implements Comparator<Object[]> {
-
-  /**
-   * mapping of dictionary dimensions and no dictionary of sort_column.
-   */
-  private boolean[] noDictionarySortColumnMaping;
-
-  /**
-   * @param noDictionarySortColumnMaping
-   */
-  public NewRowComparator(boolean[] noDictionarySortColumnMaping) {
-    this.noDictionarySortColumnMaping = noDictionarySortColumnMaping;
-  }
-
-  /**
-   * Below method will be used to compare two mdkey
-   */
-  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);
-        if (difference != 0) {
-          return difference;
-        }
-      } else {
-        int dimFieldA = (int) rowA[index];
-        int dimFieldB = (int) rowB[index];
-        diff = dimFieldA - dimFieldB;
-        if (diff != 0) {
-          return diff;
-        }
-      }
-
-      index++;
-    }
-
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/NewRowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/NewRowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/NewRowComparatorForNormalDims.java
deleted file mode 100644
index 241882e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/NewRowComparatorForNormalDims.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.sortandgroupby.sortdata;
-
-import java.util.Comparator;
-
-/**
- * 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 NewRowComparatorForNormalDims implements Comparator<Object[]> {
-  /**
-   * dimension count
-   */
-  private int numberOfSortColumns;
-
-  /**
-   * RowComparatorForNormalDims Constructor
-   *
-   * @param numberOfSortColumns
-   */
-  public NewRowComparatorForNormalDims(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 = (int)rowA[i];
-      int dimFieldB = (int)rowB[i];
-      diff = dimFieldA - dimFieldB;
-      if (diff != 0) {
-        return diff;
-      }
-    }
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/RowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/RowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/RowComparator.java
deleted file mode 100644
index 11c42a9..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/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.sortandgroupby.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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/RowComparatorForNormalDims.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/RowComparatorForNormalDims.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/RowComparatorForNormalDims.java
deleted file mode 100644
index be29bf8..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/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.sortandgroupby.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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java
deleted file mode 100644
index 71fc564..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortDataRows.java
+++ /dev/null
@@ -1,437 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.sortandgroupby.sortdata;
-
-import java.io.BufferedOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-
-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.metadata.datatype.DataType;
-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.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class SortDataRows {
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SortDataRows.class.getName());
-  /**
-   * entryCount
-   */
-  private int entryCount;
-  /**
-   * record holder array
-   */
-  private Object[][] recordHolderList;
-  /**
-   * threadStatusObserver
-   */
-  private ThreadStatusObserver threadStatusObserver;
-  /**
-   * executor service for data sort holder
-   */
-  private ExecutorService dataSorterAndWriterExecutorService;
-  /**
-   * semaphore which will used for managing sorted data object arrays
-   */
-  private Semaphore semaphore;
-
-  private SortParameters parameters;
-
-  private int sortBufferSize;
-
-  private SortIntermediateFileMerger intermediateFileMerger;
-
-  private final Object addRowsLock = new Object();
-
-  public SortDataRows(SortParameters parameters,
-      SortIntermediateFileMerger intermediateFileMerger) {
-    this.parameters = parameters;
-
-    this.intermediateFileMerger = intermediateFileMerger;
-
-    int batchSize = CarbonProperties.getInstance().getBatchSize();
-
-    this.sortBufferSize = Math.max(parameters.getSortBufferSize(), batchSize);
-    // observer of writing file in thread
-    this.threadStatusObserver = new ThreadStatusObserver();
-  }
-
-  /**
-   * This method will be used to initialize
-   */
-  public void initialize() throws CarbonSortKeyAndGroupByException {
-
-    // create holder list which will hold incoming rows
-    // size of list will be sort buffer size + 1 to avoid creation of new
-    // array in list array
-    this.recordHolderList = new Object[sortBufferSize][];
-    // Delete if any older file exists in sort temp folder
-    deleteSortLocationIfExists();
-
-    // create new sort temp directory
-    CarbonDataProcessorUtil.createLocations(parameters.getTempFileLocation());
-    this.dataSorterAndWriterExecutorService =
-        Executors.newFixedThreadPool(parameters.getNumberOfCores());
-    semaphore = new Semaphore(parameters.getNumberOfCores());
-  }
-
-  /**
-   * This method will be used to add new row
-   *
-   * @param row new row
-   * @throws CarbonSortKeyAndGroupByException problem while writing
-   */
-  public void addRow(Object[] row) throws CarbonSortKeyAndGroupByException {
-    // 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
-    int currentSize = entryCount;
-
-    if (sortBufferSize == currentSize) {
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("************ Writing to temp file ********** ");
-      }
-      intermediateFileMerger.startMergingIfPossible();
-      Object[][] recordHolderListLocal = recordHolderList;
-      try {
-        semaphore.acquire();
-        dataSorterAndWriterExecutorService.execute(new DataSorterAndWriter(recordHolderListLocal));
-      } catch (InterruptedException e) {
-        LOGGER.error(
-            "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
-        throw new CarbonSortKeyAndGroupByException(e.getMessage());
-      }
-      // create the new holder Array
-      this.recordHolderList = new Object[this.sortBufferSize][];
-      this.entryCount = 0;
-    }
-    recordHolderList[entryCount++] = row;
-  }
-
-  /**
-   * This method will be used to add new row
-   *
-   * @param rowBatch new rowBatch
-   * @throws CarbonSortKeyAndGroupByException problem while writing
-   */
-  public void addRowBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
-    // 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
-    synchronized (addRowsLock) {
-      int sizeLeft = 0;
-      if (entryCount + size >= sortBufferSize) {
-        if (LOGGER.isDebugEnabled()) {
-          LOGGER.debug("************ Writing to temp file ********** ");
-        }
-        intermediateFileMerger.startMergingIfPossible();
-        Object[][] recordHolderListLocal = recordHolderList;
-        sizeLeft = sortBufferSize - entryCount ;
-        if (sizeLeft > 0) {
-          System.arraycopy(rowBatch, 0, recordHolderListLocal, entryCount, sizeLeft);
-        }
-        try {
-          semaphore.acquire();
-          dataSorterAndWriterExecutorService
-              .execute(new DataSorterAndWriter(recordHolderListLocal));
-        } catch (Exception e) {
-          LOGGER.error(
-              "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
-          throw new CarbonSortKeyAndGroupByException(e);
-        }
-        // create the new holder Array
-        this.recordHolderList = new Object[this.sortBufferSize][];
-        this.entryCount = 0;
-        size = size - sizeLeft;
-        if (size == 0) {
-          return;
-        }
-      }
-      System.arraycopy(rowBatch, sizeLeft, recordHolderList, entryCount, size);
-      entryCount += size;
-    }
-  }
-
-  /**
-   * Below method will be used to start storing process This method will get
-   * all the temp files present in sort temp folder then it will create the
-   * record holder heap and then it will read first record from each file and
-   * initialize the heap
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  public void startSorting() throws CarbonSortKeyAndGroupByException {
-    LOGGER.info("File based sorting will be used");
-    if (this.entryCount > 0) {
-      Object[][] toSort;
-      toSort = new Object[entryCount][];
-      System.arraycopy(recordHolderList, 0, toSort, 0, entryCount);
-      if (parameters.getNumberOfNoDictSortColumns() > 0) {
-        Arrays.sort(toSort, new NewRowComparator(parameters.getNoDictionarySortColumn()));
-      } else {
-        Arrays.sort(toSort, new NewRowComparatorForNormalDims(parameters.getNumberOfSortColumns()));
-      }
-      recordHolderList = toSort;
-
-      // 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);
-      writeDataTofile(recordHolderList, this.entryCount, file);
-
-    }
-
-    startFileBasedMerge();
-    this.recordHolderList = null;
-  }
-
-  /**
-   * Below method will be used to write data to file
-   *
-   * @throws CarbonSortKeyAndGroupByException problem while writing
-   */
-  private void writeDataTofile(Object[][] recordHolderList, int entryCountLocal, File file)
-      throws CarbonSortKeyAndGroupByException {
-    // stream
-    if (parameters.isSortFileCompressionEnabled() || parameters.isPrefetch()) {
-      writeSortTempFile(recordHolderList, entryCountLocal, file);
-      return;
-    }
-    writeData(recordHolderList, entryCountLocal, file);
-  }
-
-  private void writeSortTempFile(Object[][] recordHolderList, int entryCountLocal, File file)
-      throws CarbonSortKeyAndGroupByException {
-    TempSortFileWriter writer = null;
-
-    try {
-      writer = getWriter();
-      writer.initiaize(file, entryCountLocal);
-      writer.writeSortTempFile(recordHolderList);
-    } catch (CarbonSortKeyAndGroupByException e) {
-      LOGGER.error(e, "Problem while writing the sort temp file");
-      throw e;
-    } finally {
-      if (writer != null) {
-        writer.finish();
-      }
-    }
-  }
-
-  private void writeData(Object[][] recordHolderList, int entryCountLocal, File file)
-      throws CarbonSortKeyAndGroupByException {
-    DataOutputStream stream = null;
-    try {
-      // open stream
-      stream = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(file),
-          parameters.getFileWriteBufferSize()));
-
-      // 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);
-            switch (type[mesCount]) {
-              case SHORT:
-                stream.writeShort((Short) value);
-                break;
-              case INT:
-                stream.writeInt((Integer) value);
-                break;
-              case LONG:
-                stream.writeLong((Long) value);
-                break;
-              case DOUBLE:
-                stream.writeDouble((Double) value);
-                break;
-              case DECIMAL:
-                BigDecimal val = (BigDecimal) value;
-                byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
-                stream.writeInt(bigDecimalInBytes.length);
-                stream.write(bigDecimalInBytes);
-                break;
-              default:
-                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);
-    } finally {
-      // close streams
-      CarbonUtil.closeStreams(stream);
-    }
-  }
-
-  private TempSortFileWriter getWriter() {
-    TempSortFileWriter chunkWriter = null;
-    TempSortFileWriter writer = TempSortFileWriterFactory.getInstance()
-        .getTempSortFileWriter(parameters.isSortFileCompressionEnabled(),
-            parameters.getDimColCount(), parameters.getComplexDimColCount(),
-            parameters.getMeasureColCount(), parameters.getNoDictionaryCount(),
-            parameters.getFileWriteBufferSize());
-
-    if (parameters.isPrefetch() && !parameters.isSortFileCompressionEnabled()) {
-      chunkWriter = new SortTempFileChunkWriter(writer, parameters.getBufferSize());
-    } else {
-      chunkWriter =
-          new SortTempFileChunkWriter(writer, parameters.getSortTempFileNoOFRecordsInCompression());
-    }
-
-    return chunkWriter;
-  }
-
-  /**
-   * This method will be used to delete sort temp location is it is exites
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  public void deleteSortLocationIfExists() throws CarbonSortKeyAndGroupByException {
-    CarbonDataProcessorUtil.deleteSortLocationIfExists(parameters.getTempFileLocation());
-  }
-
-  /**
-   * Below method will be used to start file based merge
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private void startFileBasedMerge() throws CarbonSortKeyAndGroupByException {
-    try {
-      dataSorterAndWriterExecutorService.shutdown();
-      dataSorterAndWriterExecutorService.awaitTermination(2, TimeUnit.DAYS);
-    } catch (InterruptedException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while shutdown the server ", e);
-    }
-  }
-
-  /**
-   * Observer class for thread execution
-   * In case of any failure we need stop all the running thread
-   */
-  private class ThreadStatusObserver {
-    /**
-     * Below method will be called if any thread fails during execution
-     *
-     * @param exception
-     * @throws CarbonSortKeyAndGroupByException
-     */
-    public void notifyFailed(Throwable exception) throws CarbonSortKeyAndGroupByException {
-      dataSorterAndWriterExecutorService.shutdownNow();
-      intermediateFileMerger.close();
-      parameters.getObserver().setFailed(true);
-      LOGGER.error(exception);
-      throw new CarbonSortKeyAndGroupByException(exception);
-    }
-  }
-
-  /**
-   * This class is responsible for sorting and writing the object
-   * array which holds the records equal to given array size
-   */
-  private class DataSorterAndWriter implements Runnable {
-    private Object[][] recordHolderArray;
-
-    public DataSorterAndWriter(Object[][] recordHolderArray) {
-      this.recordHolderArray = recordHolderArray;
-    }
-
-    @Override
-    public void run() {
-      try {
-        long startTime = System.currentTimeMillis();
-        if (parameters.getNumberOfNoDictSortColumns() > 0) {
-          Arrays.sort(recordHolderArray,
-              new NewRowComparator(parameters.getNoDictionarySortColumn()));
-        } else {
-          Arrays.sort(recordHolderArray,
-              new NewRowComparatorForNormalDims(parameters.getNumberOfSortColumns()));
-        }
-
-        // create a new file and choose folder randomly every time
-        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);
-        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
-        intermediateFileMerger.addFileToMerge(sortTempFile);
-        LOGGER.info("Time taken to sort and write sort temp file " + sortTempFile + " is: " + (
-            System.currentTimeMillis() - startTime));
-      } catch (Throwable e) {
-        try {
-          threadStatusObserver.notifyFailed(e);
-        } catch (CarbonSortKeyAndGroupByException ex) {
-          LOGGER.error(ex);
-        }
-      } finally {
-        semaphore.release();
-      }
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortIntermediateFileMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortIntermediateFileMerger.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortIntermediateFileMerger.java
deleted file mode 100644
index 6bda88a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortIntermediateFileMerger.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.sortandgroupby.sortdata;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-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.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-/**
- * It does mergesort intermediate files to big file.
- */
-public class SortIntermediateFileMerger {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SortIntermediateFileMerger.class.getName());
-
-  /**
-   * executorService
-   */
-  private ExecutorService executorService;
-  /**
-   * procFiles
-   */
-  private List<File> procFiles;
-
-  private SortParameters parameters;
-
-  private final Object lockObject = new Object();
-
-  public SortIntermediateFileMerger(SortParameters parameters) {
-    this.parameters = parameters;
-    // processed file list
-    this.procFiles = new ArrayList<File>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    this.executorService = Executors.newFixedThreadPool(parameters.getNumberOfCores());
-  }
-
-  public void addFileToMerge(File sortTempFile) {
-    // add sort temp filename to and arrayList. When the list size reaches 20 then
-    // intermediate merging of sort temp files will be triggered
-    synchronized (lockObject) {
-      procFiles.add(sortTempFile);
-    }
-  }
-
-  public void startMergingIfPossible() {
-    File[] fileList;
-    if (procFiles.size() >= parameters.getNumberOfIntermediateFileToBeMerged()) {
-      synchronized (lockObject) {
-        fileList = procFiles.toArray(new File[procFiles.size()]);
-        this.procFiles = new ArrayList<File>();
-      }
-      if (LOGGER.isDebugEnabled()) {
-        LOGGER.debug("Sumitting request for intermediate merging no of files: " + fileList.length);
-      }
-      startIntermediateMerging(fileList);
-    }
-  }
-
-  /**
-   * Below method will be used to start the intermediate file merging
-   *
-   * @param intermediateFiles
-   */
-  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);
-    IntermediateFileMerger merger = new IntermediateFileMerger(parameters, intermediateFiles, file);
-    executorService.execute(merger);
-  }
-
-  public void finish() throws CarbonSortKeyAndGroupByException {
-    try {
-      executorService.shutdown();
-      executorService.awaitTermination(2, TimeUnit.DAYS);
-    } catch (InterruptedException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while shutdown the server ", e);
-    }
-    procFiles.clear();
-    procFiles = null;
-  }
-
-  public void close() {
-    if (executorService.isShutdown()) {
-      executorService.shutdownNow();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortParameters.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortParameters.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortParameters.java
deleted file mode 100644
index fb2977e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortParameters.java
+++ /dev/null
@@ -1,603 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.sortandgroupby.sortdata;
-
-import java.io.File;
-import java.io.Serializable;
-
-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.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.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.schema.metadata.SortObserver;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import org.apache.commons.lang3.StringUtils;
-
-public class SortParameters implements Serializable {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SortParameters.class.getName());
-  /**
-   * tempFileLocation
-   */
-  private String[] tempFileLocation;
-  /**
-   * sortBufferSize
-   */
-  private int sortBufferSize;
-  /**
-   * measure count
-   */
-  private int measureColCount;
-  /**
-   * measure count
-   */
-  private int dimColCount;
-  /**
-   * measure count
-   */
-  private int complexDimColCount;
-  /**
-   * fileBufferSize
-   */
-  private int fileBufferSize;
-  /**
-   * numberOfIntermediateFileToBeMerged
-   */
-  private int numberOfIntermediateFileToBeMerged;
-  /**
-   * fileWriteBufferSize
-   */
-  private int fileWriteBufferSize;
-  /**
-   * observer
-   */
-  private SortObserver observer;
-  /**
-   * sortTempFileNoOFRecordsInCompression
-   */
-  private int sortTempFileNoOFRecordsInCompression;
-  /**
-   * isSortTempFileCompressionEnabled
-   */
-  private boolean isSortFileCompressionEnabled;
-  /**
-   * prefetch
-   */
-  private boolean prefetch;
-  /**
-   * bufferSize
-   */
-  private int bufferSize;
-
-  private String databaseName;
-
-  private String tableName;
-
-  private DataType[] measureDataType;
-
-  /**
-   * To know how many columns are of high cardinality.
-   */
-  private int noDictionaryCount;
-  /**
-   * partitionID
-   */
-  private String partitionID;
-  /**
-   * Id of the load folder
-   */
-  private String segmentId;
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-
-  /**
-   * This will tell whether dimension is dictionary or not.
-   */
-  private boolean[] noDictionaryDimnesionColumn;
-
-  private boolean[] noDictionarySortColumn;
-
-  private int numberOfSortColumns;
-
-  private int numberOfNoDictSortColumns;
-
-  private int numberOfCores;
-
-  private int batchSortSizeinMb;
-
-  public SortParameters getCopy() {
-    SortParameters parameters = new SortParameters();
-    parameters.tempFileLocation = tempFileLocation;
-    parameters.sortBufferSize = sortBufferSize;
-    parameters.measureColCount = measureColCount;
-    parameters.dimColCount = dimColCount;
-    parameters.complexDimColCount = complexDimColCount;
-    parameters.fileBufferSize = fileBufferSize;
-    parameters.numberOfIntermediateFileToBeMerged = numberOfIntermediateFileToBeMerged;
-    parameters.fileWriteBufferSize = fileWriteBufferSize;
-    parameters.observer = observer;
-    parameters.sortTempFileNoOFRecordsInCompression = sortTempFileNoOFRecordsInCompression;
-    parameters.isSortFileCompressionEnabled = isSortFileCompressionEnabled;
-    parameters.prefetch = prefetch;
-    parameters.bufferSize = bufferSize;
-    parameters.databaseName = databaseName;
-    parameters.tableName = tableName;
-    parameters.measureDataType = measureDataType;
-    parameters.noDictionaryCount = noDictionaryCount;
-    parameters.partitionID = partitionID;
-    parameters.segmentId = segmentId;
-    parameters.taskNo = taskNo;
-    parameters.noDictionaryDimnesionColumn = noDictionaryDimnesionColumn;
-    parameters.noDictionarySortColumn = noDictionarySortColumn;
-    parameters.numberOfSortColumns = numberOfSortColumns;
-    parameters.numberOfNoDictSortColumns = numberOfNoDictSortColumns;
-    parameters.numberOfCores = numberOfCores;
-    parameters.batchSortSizeinMb = batchSortSizeinMb;
-    return parameters;
-  }
-
-  public String[] getTempFileLocation() {
-    return tempFileLocation;
-  }
-
-  public void setTempFileLocation(String[] tempFileLocation) {
-    this.tempFileLocation = tempFileLocation;
-  }
-
-  public int getSortBufferSize() {
-    return sortBufferSize;
-  }
-
-  public void setSortBufferSize(int sortBufferSize) {
-    this.sortBufferSize = sortBufferSize;
-  }
-
-  public int getMeasureColCount() {
-    return measureColCount;
-  }
-
-  public void setMeasureColCount(int measureColCount) {
-    this.measureColCount = measureColCount;
-  }
-
-  public int getDimColCount() {
-    return dimColCount;
-  }
-
-  public void setDimColCount(int dimColCount) {
-    this.dimColCount = dimColCount;
-  }
-
-  public int getComplexDimColCount() {
-    return complexDimColCount;
-  }
-
-  public void setComplexDimColCount(int complexDimColCount) {
-    this.complexDimColCount = complexDimColCount;
-  }
-
-  public int getFileBufferSize() {
-    return fileBufferSize;
-  }
-
-  public void setFileBufferSize(int fileBufferSize) {
-    this.fileBufferSize = fileBufferSize;
-  }
-
-  public int getNumberOfIntermediateFileToBeMerged() {
-    return numberOfIntermediateFileToBeMerged;
-  }
-
-  public void setNumberOfIntermediateFileToBeMerged(int numberOfIntermediateFileToBeMerged) {
-    this.numberOfIntermediateFileToBeMerged = numberOfIntermediateFileToBeMerged;
-  }
-
-  public int getFileWriteBufferSize() {
-    return fileWriteBufferSize;
-  }
-
-  public void setFileWriteBufferSize(int fileWriteBufferSize) {
-    this.fileWriteBufferSize = fileWriteBufferSize;
-  }
-
-  public SortObserver getObserver() {
-    return observer;
-  }
-
-  public void setObserver(SortObserver observer) {
-    this.observer = observer;
-  }
-
-  public int getSortTempFileNoOFRecordsInCompression() {
-    return sortTempFileNoOFRecordsInCompression;
-  }
-
-  public void setSortTempFileNoOFRecordsInCompression(int sortTempFileNoOFRecordsInCompression) {
-    this.sortTempFileNoOFRecordsInCompression = sortTempFileNoOFRecordsInCompression;
-  }
-
-  public boolean isSortFileCompressionEnabled() {
-    return isSortFileCompressionEnabled;
-  }
-
-  public void setSortFileCompressionEnabled(boolean sortFileCompressionEnabled) {
-    isSortFileCompressionEnabled = sortFileCompressionEnabled;
-  }
-
-  public boolean isPrefetch() {
-    return prefetch;
-  }
-
-  public void setPrefetch(boolean prefetch) {
-    this.prefetch = prefetch;
-  }
-
-  public int getBufferSize() {
-    return bufferSize;
-  }
-
-  public void setBufferSize(int bufferSize) {
-    this.bufferSize = bufferSize;
-  }
-
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  public DataType[] getMeasureDataType() {
-    return measureDataType;
-  }
-
-  public void setMeasureDataType(DataType[] measureDataType) {
-    this.measureDataType = measureDataType;
-  }
-
-  public int getNoDictionaryCount() {
-    return noDictionaryCount;
-  }
-
-  public void setNoDictionaryCount(int noDictionaryCount) {
-    this.noDictionaryCount = noDictionaryCount;
-  }
-
-  public String getPartitionID() {
-    return partitionID;
-  }
-
-  public void setPartitionID(String partitionID) {
-    this.partitionID = partitionID;
-  }
-
-  public String getSegmentId() {
-    return segmentId;
-  }
-
-  public void setSegmentId(String segmentId) {
-    this.segmentId = segmentId;
-  }
-
-  public String getTaskNo() {
-    return taskNo;
-  }
-
-  public void setTaskNo(String taskNo) {
-    this.taskNo = taskNo;
-  }
-
-  public boolean[] getNoDictionaryDimnesionColumn() {
-    return noDictionaryDimnesionColumn;
-  }
-
-  public void setNoDictionaryDimnesionColumn(boolean[] noDictionaryDimnesionColumn) {
-    this.noDictionaryDimnesionColumn = noDictionaryDimnesionColumn;
-  }
-
-  public int getNumberOfCores() {
-    return numberOfCores;
-  }
-
-  public void setNumberOfCores(int numberOfCores) {
-    this.numberOfCores = numberOfCores;
-  }
-
-  public int getNumberOfSortColumns() {
-    return numberOfSortColumns;
-  }
-
-  public void setNumberOfSortColumns(int numberOfSortColumns) {
-    this.numberOfSortColumns = Math.min(numberOfSortColumns, this.dimColCount);
-  }
-
-  public boolean[] getNoDictionarySortColumn() {
-    return noDictionarySortColumn;
-  }
-
-  public void setNoDictionarySortColumn(boolean[] noDictionarySortColumn) {
-    this.noDictionarySortColumn = noDictionarySortColumn;
-  }
-
-  public int getNumberOfNoDictSortColumns() {
-    return numberOfNoDictSortColumns;
-  }
-
-  public void setNumberOfNoDictSortColumns(int numberOfNoDictSortColumns) {
-    this.numberOfNoDictSortColumns = Math.min(numberOfNoDictSortColumns, noDictionaryCount);
-  }
-
-  public int getBatchSortSizeinMb() {
-    return batchSortSizeinMb;
-  }
-
-  public void setBatchSortSizeinMb(int batchSortSizeinMb) {
-    this.batchSortSizeinMb = batchSortSizeinMb;
-  }
-
-  public static SortParameters createSortParameters(CarbonDataLoadConfiguration configuration) {
-    SortParameters parameters = new SortParameters();
-    CarbonTableIdentifier tableIdentifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    CarbonProperties carbonProperties = CarbonProperties.getInstance();
-    parameters.setDatabaseName(tableIdentifier.getDatabaseName());
-    parameters.setTableName(tableIdentifier.getTableName());
-    parameters.setPartitionID(configuration.getPartitionId());
-    parameters.setSegmentId(configuration.getSegmentId());
-    parameters.setTaskNo(configuration.getTaskNo());
-    parameters.setMeasureColCount(configuration.getMeasureCount());
-    parameters.setDimColCount(
-        configuration.getDimensionCount() - configuration.getComplexColumnCount());
-    parameters.setNoDictionaryCount(configuration.getNoDictionaryCount());
-    parameters.setComplexDimColCount(configuration.getComplexColumnCount());
-    parameters.setNoDictionaryDimnesionColumn(
-        CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields()));
-    parameters.setBatchSortSizeinMb(CarbonDataProcessorUtil.getBatchSortSizeinMb(configuration));
-
-    parameters.setNumberOfSortColumns(configuration.getNumberOfSortColumns());
-    parameters.setNumberOfNoDictSortColumns(configuration.getNumberOfNoDictSortColumns());
-    setNoDictionarySortColumnMapping(parameters);
-    parameters.setObserver(new SortObserver());
-    // get sort buffer size
-    parameters.setSortBufferSize(Integer.parseInt(carbonProperties
-        .getProperty(CarbonCommonConstants.SORT_SIZE,
-            CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL)));
-    LOGGER.info("Sort size for table: " + parameters.getSortBufferSize());
-    // set number of intermedaite file to merge
-    parameters.setNumberOfIntermediateFileToBeMerged(Integer.parseInt(carbonProperties
-        .getProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT,
-            CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE)));
-
-    LOGGER.info("Number of intermediate file to be merged: " + parameters
-        .getNumberOfIntermediateFileToBeMerged());
-
-    // get file buffer size
-    parameters.setFileBufferSize(CarbonDataProcessorUtil
-        .getFileBufferSize(parameters.getNumberOfIntermediateFileToBeMerged(), carbonProperties,
-            CarbonCommonConstants.CONSTANT_SIZE_TEN));
-
-    LOGGER.info("File Buffer Size: " + parameters.getFileBufferSize());
-
-    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(tableIdentifier.getDatabaseName(),
-            tableIdentifier.getTableName(), configuration.getTaskNo(),
-            configuration.getPartitionId(), configuration.getSegmentId(), false, false);
-    String[] sortTempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
-        File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
-
-    parameters.setTempFileLocation(sortTempDirs);
-    LOGGER.info("temp file location: " + StringUtils.join(parameters.getTempFileLocation(), ","));
-
-    int numberOfCores;
-    try {
-      numberOfCores = Integer.parseInt(carbonProperties
-          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-      numberOfCores = numberOfCores / 2;
-    } catch (NumberFormatException exc) {
-      numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-    }
-    parameters.setNumberOfCores(numberOfCores > 0 ? numberOfCores : 1);
-
-    parameters.setFileWriteBufferSize(Integer.parseInt(carbonProperties
-        .getProperty(CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE,
-            CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE)));
-
-    parameters.setSortFileCompressionEnabled(Boolean.parseBoolean(carbonProperties
-        .getProperty(CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED,
-            CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED_DEFAULTVALUE)));
-
-    int sortTempFileNoOFRecordsInCompression;
-    try {
-      sortTempFileNoOFRecordsInCompression = Integer.parseInt(carbonProperties
-          .getProperty(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION,
-              CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE));
-      if (sortTempFileNoOFRecordsInCompression < 1) {
-        LOGGER.error("Invalid value for: "
-            + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
-            + ":Only Positive Integer value(greater than zero) is allowed.Default value will "
-            + "be used");
-
-        sortTempFileNoOFRecordsInCompression = Integer.parseInt(
-            CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.error(
-          "Invalid value for: " + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
-              + ", only Positive Integer value is allowed. Default value will be used");
-
-      sortTempFileNoOFRecordsInCompression = Integer
-          .parseInt(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
-    }
-    parameters.setSortTempFileNoOFRecordsInCompression(sortTempFileNoOFRecordsInCompression);
-
-    if (parameters.isSortFileCompressionEnabled()) {
-      LOGGER.info("Compression will be used for writing the sort temp File");
-    }
-
-    parameters.setPrefetch(CarbonCommonConstants.CARBON_PREFETCH_IN_MERGE_VALUE);
-    parameters.setBufferSize(Integer.parseInt(carbonProperties.getProperty(
-        CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
-        CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT)));
-
-    DataType[] measureDataType = configuration.getMeasureDataType();
-    parameters.setMeasureDataType(measureDataType);
-    return parameters;
-  }
-
-  /**
-   * this method will set the boolean mapping for no dictionary sort columns
-   *
-   * @param parameters
-   */
-  private static void setNoDictionarySortColumnMapping(SortParameters parameters) {
-    if (parameters.getNumberOfSortColumns() == parameters.getNoDictionaryDimnesionColumn().length) {
-      parameters.setNoDictionarySortColumn(parameters.getNoDictionaryDimnesionColumn());
-    } else {
-      boolean[] noDictionarySortColumnTemp = new boolean[parameters.getNumberOfSortColumns()];
-      System
-          .arraycopy(parameters.getNoDictionaryDimnesionColumn(), 0, noDictionarySortColumnTemp, 0,
-              parameters.getNumberOfSortColumns());
-      parameters.setNoDictionarySortColumn(noDictionarySortColumnTemp);
-    }
-  }
-
-  public static SortParameters createSortParameters(CarbonTable carbonTable, String databaseName,
-      String tableName, int dimColCount, int complexDimColCount, int measureColCount,
-      int noDictionaryCount, String partitionID, 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.setSegmentId(segmentId);
-    parameters.setTaskNo(taskNo);
-    parameters.setMeasureColCount(measureColCount);
-    parameters.setDimColCount(dimColCount - complexDimColCount);
-    parameters.setNumberOfSortColumns(carbonTable.getNumberOfSortColumns());
-    parameters.setNoDictionaryCount(noDictionaryCount);
-    parameters.setNumberOfNoDictSortColumns(carbonTable.getNumberOfNoDictSortColumns());
-    parameters.setComplexDimColCount(complexDimColCount);
-    parameters.setNoDictionaryDimnesionColumn(noDictionaryColMaping);
-    parameters.setObserver(new SortObserver());
-    // get sort buffer size
-    parameters.setSortBufferSize(Integer.parseInt(carbonProperties
-        .getProperty(CarbonCommonConstants.SORT_SIZE,
-            CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL)));
-    LOGGER.info("Sort size for table: " + parameters.getSortBufferSize());
-    // set number of intermedaite file to merge
-    parameters.setNumberOfIntermediateFileToBeMerged(Integer.parseInt(carbonProperties
-        .getProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT,
-            CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE)));
-
-    LOGGER.info("Number of intermediate file to be merged: " + parameters
-        .getNumberOfIntermediateFileToBeMerged());
-
-    // get file buffer size
-    parameters.setFileBufferSize(CarbonDataProcessorUtil
-        .getFileBufferSize(parameters.getNumberOfIntermediateFileToBeMerged(), carbonProperties,
-            CarbonCommonConstants.CONSTANT_SIZE_TEN));
-
-    LOGGER.info("File Buffer Size: " + parameters.getFileBufferSize());
-
-    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(databaseName, tableName, taskNo, partitionID, segmentId,
-            isCompactionFlow, false);
-    String[] sortTempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
-        File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
-    parameters.setTempFileLocation(sortTempDirs);
-    LOGGER.info("temp file location: " + StringUtils.join(parameters.getTempFileLocation(), ","));
-
-    int numberOfCores;
-    try {
-      numberOfCores = Integer.parseInt(carbonProperties
-          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-      numberOfCores = numberOfCores / 2;
-    } catch (NumberFormatException exc) {
-      numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-    }
-    parameters.setNumberOfCores(numberOfCores > 0 ? numberOfCores : 1);
-
-    parameters.setFileWriteBufferSize(Integer.parseInt(carbonProperties
-        .getProperty(CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE,
-            CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE)));
-
-    parameters.setSortFileCompressionEnabled(Boolean.parseBoolean(carbonProperties
-        .getProperty(CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED,
-            CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED_DEFAULTVALUE)));
-
-    int sortTempFileNoOFRecordsInCompression;
-    try {
-      sortTempFileNoOFRecordsInCompression = Integer.parseInt(carbonProperties
-          .getProperty(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION,
-              CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE));
-      if (sortTempFileNoOFRecordsInCompression < 1) {
-        LOGGER.error("Invalid value for: "
-            + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
-            + ":Only Positive Integer value(greater than zero) is allowed.Default value will "
-            + "be used");
-
-        sortTempFileNoOFRecordsInCompression = Integer.parseInt(
-            CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.error(
-          "Invalid value for: " + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
-              + ", only Positive Integer value is allowed. Default value will be used");
-
-      sortTempFileNoOFRecordsInCompression = Integer
-          .parseInt(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
-    }
-    parameters.setSortTempFileNoOFRecordsInCompression(sortTempFileNoOFRecordsInCompression);
-
-    if (parameters.isSortFileCompressionEnabled()) {
-      LOGGER.info("Compression will be used for writing the sort temp File");
-    }
-
-    parameters.setPrefetch(CarbonCommonConstants. CARBON_PREFETCH_IN_MERGE_VALUE);
-    parameters.setBufferSize(Integer.parseInt(carbonProperties.getProperty(
-        CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
-        CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT)));
-
-    DataType[] type = CarbonDataProcessorUtil
-        .getMeasureDataType(parameters.getMeasureColCount(), parameters.getDatabaseName(),
-            parameters.getTableName());
-    parameters.setMeasureDataType(type);
-    setNoDictionarySortColumnMapping(parameters);
-    return parameters;
-  }
-
-}


[45/50] [abbrv] carbondata git commit: [CARBONDATA-1547] Table should be dropped after running test

Posted by ja...@apache.org.
[CARBONDATA-1547] Table should be dropped after running test

Table should be dropped after running test, this PR adds table dropping in afterAll in all testcases

This closes #1405


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

Branch: refs/heads/streaming_ingest
Commit: ccdcc3ccd207c8a485db94937ea54ad39d6c131d
Parents: a7c36ef
Author: xubo245 <60...@qq.com>
Authored: Mon Oct 9 21:27:33 2017 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Oct 10 10:25:08 2017 +0800

----------------------------------------------------------------------
 .../spark/testsuite/complexType/TestComplexTypeQuery.scala  | 3 ++-
 .../allqueries/InsertIntoCarbonTableTestCase.scala          | 9 ++++++++-
 .../dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala  | 1 +
 .../carbondata/spark/testsuite/datetype/DateTypeTest.scala  | 7 ++++++-
 .../testsuite/detailquery/ExpressionWithNullTestCase.scala  | 7 +++++++
 .../detailquery/RangeFilterAllDataTypesTestCases.scala      | 4 +---
 .../spark/testsuite/detailquery/RangeFilterTestCase.scala   | 3 +--
 .../spark/testsuite/iud/HorizontalCompactionTestCase.scala  | 1 +
 .../spark/testsuite/partition/TestAlterPartitionTable.scala | 1 +
 .../org/apache/spark/carbondata/CarbonDataSourceSuite.scala | 2 ++
 .../spark/carbondata/bucketing/TableBucketingTestCase.scala | 2 ++
 .../carbondata/restructure/AlterTableRevertTestCase.scala   | 2 ++
 .../restructure/AlterTableValidationTestCase.scala          | 2 ++
 13 files changed, 36 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexTypeQuery.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexTypeQuery.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexTypeQuery.scala
index 3cd5354..1f66f26 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexTypeQuery.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestComplexTypeQuery.scala
@@ -286,6 +286,7 @@ class TestComplexTypeQuery extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists structusingstructHive")
     sql("drop table if exists structusingarraycarbon")
     sql("drop table if exists structusingarrayhive")
-
+    sql("drop table if exists complex_filter")
+    sql("drop table if exists carbon_table")
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/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 8a084ae..34981a2 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
@@ -35,6 +35,7 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("create table THive (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions st
 ring, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','")
     sql(s"LOAD DATA local INPATH '$resourcesPath/100_olap.csv' INTO TABLE THive")
   }
+
   test("insert from hive") {
      sql("drop table if exists TCarbon")
      sql("create table TCarbon (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_phonePADPartitionedVersions
  string, Latest_YEAR int, Latest_MONTH int, Latest_DAY Decimal(30,10), Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string,gamePointId double,contractNumber BigInt) STORED BY 'org.apache.carbondata.format'")
@@ -44,6 +45,7 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
          sql("select imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_oper
 atorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription from TCarbon order by imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,La
 test_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription")
      )
   }
+
   test("insert from carbon-select columns") {
      sql("drop table if exists TCarbonSource")
      sql("drop table if exists TCarbon")
@@ -265,7 +267,6 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     assert(rowCount == sql("select imei from TCarbonSourceOverwrite").count())
   }
 
-
   override def afterAll {
     sql("drop table if exists load")
     sql("drop table if exists inser")
@@ -279,6 +280,12 @@ class InsertIntoCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists HiveDest")
     sql("drop table if exists CarbonOverwrite")
     sql("drop table if exists HiveOverwrite")
+    sql("drop table if exists tcarbonsourceoverwrite")
+    sql("drop table if exists carbon_table1")
+    sql("drop table if exists carbon_table")
+    sql("DROP TABLE IF EXISTS student")
+    sql("DROP TABLE IF EXISTS uniqdata")
+
     if (timeStampPropOrig != null) {
       CarbonProperties.getInstance()
         .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, timeStampPropOrig)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala
index 9416d47..c79871f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntaxDefaultFormat.scala
@@ -708,5 +708,6 @@ class TestLoadDataWithHiveSyntaxDefaultFormat extends QueryTest with BeforeAndAf
     sql("drop table if exists carbontable1")
     sql("drop table if exists hivetable1")
     sql("drop table if exists comment_test")
+    sql("drop table if exists double_test")
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/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 a71cee1..e2df07c 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
@@ -18,10 +18,15 @@ 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
 
 
-class DateTypeTest extends QueryTest {
+class DateTypeTest extends QueryTest with BeforeAndAfterAll{
 
+  override def afterAll: Unit = {
+    sql("DROP TABLE IF EXISTS Carbon_automation_testdate")
+    sql("DROP TABLE IF EXISTS Carbon_automation_testtimestamp")
+  }
 
   test("must throw exception for date data type in dictionary_exclude") {
     try {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ExpressionWithNullTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ExpressionWithNullTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ExpressionWithNullTestCase.scala
index 79dbf6b..2024018 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ExpressionWithNullTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/ExpressionWithNullTestCase.scala
@@ -37,6 +37,13 @@ class ExpressionWithNullTestCase extends QueryTest with BeforeAndAfterAll {
     sql(s"load data local inpath '$resourcesPath/filter/datawithoutnull.csv' into table expression_hive")
   }
 
+  override def afterAll = {
+    sql("drop table if exists expression_test")
+    sql("drop table if exists expression_test_hive")
+    sql("drop table if exists expression")
+    sql("drop table if exists expression_hive")
+  }
+
   test("test to check in expression with null values") {
     checkAnswer(sql("select * from expression_test where id in (1,2,'', NULL, ' ')"), sql("select * from expression_test_hive where id in (1,2,' ', NULL, ' ')"))
     checkAnswer(sql("select * from expression_test where id in (1,2,'')"), sql("select * from expression_test_hive where id in (1,2,'')"))

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala
index ac9325d..00be125 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterAllDataTypesTestCases.scala
@@ -647,9 +647,6 @@ class RangeFilterMyTests extends QueryTest with BeforeAndAfterAll {
     )
   }
 
-
-
-
   override def afterAll {
     sql("drop table if exists filtertestTable")
     sql("drop table if exists NO_DICTIONARY_HIVE_1")
@@ -668,6 +665,7 @@ class RangeFilterMyTests extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists NO_DICTIONARY_CARBON_7")
     sql("drop table if exists NO_DICTIONARY_CARBON_8")
     sql("drop table if exists NO_DICTIONARY_HIVE_8")
+    sql("drop table if exists directdictionarytable_hive")
     //sql("drop cube NO_DICTIONARY_CARBON_1")
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala
index ea9c481..cd7f7fc 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/RangeFilterTestCase.scala
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.spark.testsuite.detailquery
 
-import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
@@ -588,7 +587,6 @@ class RangeFilterTestCase extends QueryTest with BeforeAndAfterAll {
       sql("select empname from NO_DICTIONARY_HIVE_8 where empname <= '107'"))
   }
 
-
   override def afterAll {
     sql("drop table if exists filtertestTable")
     sql("drop table if exists NO_DICTIONARY_HIVE_1")
@@ -599,5 +597,6 @@ class RangeFilterTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists NO_DICTIONARY_CARBON_6")
     sql("drop table if exists DICTIONARY_CARBON_6")
     sql("drop table if exists NO_DICTIONARY_CARBON_7")
+    sql("drop table if exists NO_DICTIONARY_HIVE_8")
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala
index 48f9fde..9d4b708 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala
@@ -359,6 +359,7 @@ class HorizontalCompactionTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop database if exists iud4 cascade")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.isHorizontalCompactionEnabled , "true")
+    sql("""drop table if exists t_carbn01""")
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/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 a4a431d..5e81044 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
@@ -840,6 +840,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS test_range_date")
     sql("DROP TABLE IF EXISTS test_range_timestamp")
     sql("DROP TABLE IF EXISTS test_range_decimal")
+    sql("drop table if exists test_invalid_partition_id")
   }
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
index 7849485..d2571b7 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/CarbonDataSourceSuite.scala
@@ -64,6 +64,8 @@ class CarbonDataSourceSuite extends Spark2QueryTest with BeforeAndAfterAll {
   override def afterAll(): Unit = {
     sql("drop table carbon_testtable")
     sql("DROP TABLE IF EXISTS csv_table")
+    sql("DROP TABLE IF EXISTS car")
+    sql("drop table if exists sparkunion")
   }
 
   test("project") {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/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 d30b3d1..cbc7879 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
@@ -221,6 +221,8 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS t8")
     sql("DROP TABLE IF EXISTS t9")
     sql("DROP TABLE IF EXISTS t10")
+    sql("DROP TABLE IF EXISTS bucketed_parquet_table")
+    sql("DROP TABLE IF EXISTS parquet_table")
     sqlContext.setConf("spark.sql.autoBroadcastJoinThreshold", threshold.toString)
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/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 c248e31..3fcad74 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
@@ -58,6 +58,7 @@ class AlterTableRevertTestCase extends Spark2QueryTest with BeforeAndAfterAll {
     }
     val result = sql("select * from reverttest").count()
     assert(result.equals(1L))
+    sql("drop table if exists reverttest_fail")
   }
 
   test("test to revert drop columns on failure") {
@@ -109,6 +110,7 @@ class AlterTableRevertTestCase extends Spark2QueryTest with BeforeAndAfterAll {
   override def afterAll() {
     hiveClient.runSqlHive("set hive.security.authorization.enabled=false")
     sql("drop table if exists reverttest")
+    sql("drop table if exists reverttest_fail")
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ccdcc3cc/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
index 3dab247..9cfb77a 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/AlterTableValidationTestCase.scala
@@ -462,6 +462,7 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
     checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"SORT_COLUMNS")
     checkExistence(sql("describe formatted specifiedSortColumnsWithAlter"),true,"empno,empname,role,doj")
   }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS restructure")
     sql("drop table if exists table1")
@@ -471,6 +472,7 @@ class AlterTableValidationTestCase extends Spark2QueryTest with BeforeAndAfterAl
     sql("DROP TABLE IF EXISTS restructure_badnew")
     sql("DROP TABLE IF EXISTS lock_rename")
     sql("drop table if exists uniqdata")
+    sql("drop table if exists uniqdata1")
     sql("drop table if exists defaultSortColumnsWithAlter")
     sql("drop table if exists specifiedSortColumnsWithAlter")
   }


[14/50] [abbrv] carbondata git commit: [CARBONDATA-1529] Resolved the partition Table link in README.md

Posted by ja...@apache.org.
[CARBONDATA-1529] Resolved the partition Table link in README.md

This closes #1389


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

Branch: refs/heads/streaming_ingest
Commit: d304ef96ec59af6259a470ff35be85cf1283f326
Parents: d5dd78a
Author: PallaviSingh1992 <pa...@yahoo.co.in>
Authored: Wed Sep 27 16:16:05 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Wed Sep 27 22:32:50 2017 +0800

----------------------------------------------------------------------
 README.md | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d304ef96/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 7033238..06f1ce5 100644
--- a/README.md
+++ b/README.md
@@ -48,7 +48,7 @@ CarbonData is built using Apache Maven, to [build CarbonData](https://github.com
 * [Data Management](https://github.com/apache/carbondata/blob/master/docs/data-management.md)
 * [DDL Operations on CarbonData](https://github.com/apache/carbondata/blob/master/docs/ddl-operation-on-carbondata.md) 
 * [DML Operations on CarbonData](https://github.com/apache/carbondata/blob/master/docs/dml-operation-on-carbondata.md)  
-* [Partition Table](https://https://github.com/apache/carbondata/blob/master/docs/partition-guide.md)
+* [Partition Table](https://github.com/apache/carbondata/blob/master/docs/partition-guide.md)
 * [Cluster Installation and Deployment](https://github.com/apache/carbondata/blob/master/docs/installation-guide.md)
 * [Configuring Carbondata](https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.md)
 * [FAQ](https://github.com/apache/carbondata/blob/master/docs/faq.md)


[27/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterProcessorStepImpl.java
deleted file mode 100644
index c7af420..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterProcessorStepImpl.java
+++ /dev/null
@@ -1,199 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.steps;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-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.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.processing.newflow.AbstractDataLoadProcessorStep;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
-import org.apache.carbondata.processing.store.CarbonFactHandler;
-import org.apache.carbondata.processing.store.CarbonFactHandlerFactory;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * It reads data from sorted files which are generated in previous sort step.
- * And it writes data to carbondata file. It also generates mdk key while writing to carbondata file
- */
-public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DataWriterProcessorStepImpl.class.getName());
-
-  private long readCounter;
-
-  public DataWriterProcessorStepImpl(CarbonDataLoadConfiguration configuration,
-      AbstractDataLoadProcessorStep child) {
-    super(configuration, child);
-  }
-
-  public DataWriterProcessorStepImpl(CarbonDataLoadConfiguration configuration) {
-    super(configuration, null);
-  }
-
-  @Override public DataField[] getOutput() {
-    return child.getOutput();
-  }
-
-  @Override public void initialize() throws IOException {
-    super.initialize();
-    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;
-  }
-
-  public CarbonFactDataHandlerModel getDataHandlerModel(int partitionId) {
-    CarbonTableIdentifier tableIdentifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    String[] storeLocation = getStoreLocation(tableIdentifier, String.valueOf(partitionId));
-    CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
-        .createCarbonFactDataHandlerModel(configuration, storeLocation, partitionId, 0);
-    return model;
-  }
-
-  @Override public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
-    Iterator<CarbonRowBatch>[] iterators = child.execute();
-    CarbonTableIdentifier tableIdentifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    String tableName = tableIdentifier.getTableName();
-    try {
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
-              System.currentTimeMillis());
-      int i = 0;
-      for (Iterator<CarbonRowBatch> iterator : iterators) {
-        String[] storeLocation = getStoreLocation(tableIdentifier, String.valueOf(i));
-
-        CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
-            .createCarbonFactDataHandlerModel(configuration, storeLocation, i, 0);
-        CarbonFactHandler dataHandler = null;
-        boolean rowsNotExist = true;
-        while (iterator.hasNext()) {
-          if (rowsNotExist) {
-            rowsNotExist = false;
-            dataHandler = CarbonFactHandlerFactory
-                .createCarbonFactHandler(model, CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
-            dataHandler.initialise();
-          }
-          processBatch(iterator.next(), dataHandler);
-        }
-        if (!rowsNotExist) {
-          finish(dataHandler);
-        }
-        i++;
-      }
-
-    } catch (CarbonDataWriterException e) {
-      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
-      throw new CarbonDataLoadingException(
-          "Error while initializing data handler : " + e.getMessage());
-    } catch (Exception e) {
-      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
-      throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage(), e);
-    }
-    return null;
-  }
-
-  @Override protected String getStepName() {
-    return "Data Writer";
-  }
-
-  public void finish(CarbonFactHandler dataHandler) {
-    CarbonTableIdentifier tableIdentifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    String tableName = tableIdentifier.getTableName();
-
-    try {
-      dataHandler.finish();
-    } catch (Exception e) {
-      LOGGER.error(e, "Failed for table: " + tableName + " in  finishing data handler");
-    }
-    LOGGER.info("Record Processed For table: " + tableName);
-    String logMessage =
-        "Finished Carbon DataWriterProcessorStepImpl: Read: " + readCounter + ": Write: "
-            + rowCounter.get();
-    LOGGER.info(logMessage);
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
-    processingComplete(dataHandler);
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
-            System.currentTimeMillis());
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
-  }
-
-  private void processingComplete(CarbonFactHandler dataHandler) throws CarbonDataLoadingException {
-    if (null != dataHandler) {
-      try {
-        dataHandler.closeHandler();
-      } catch (CarbonDataWriterException e) {
-        LOGGER.error(e, e.getMessage());
-        throw new CarbonDataLoadingException(e.getMessage(), e);
-      } catch (Exception e) {
-        LOGGER.error(e, e.getMessage());
-        throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage());
-      }
-    }
-  }
-
-  private void processBatch(CarbonRowBatch batch, CarbonFactHandler dataHandler)
-      throws CarbonDataLoadingException {
-    try {
-      while (batch.hasNext()) {
-        CarbonRow row = batch.next();
-        dataHandler.addDataToStore(row);
-        readCounter++;
-      }
-    } catch (Exception e) {
-      throw new CarbonDataLoadingException(e);
-    }
-    rowCounter.getAndAdd(batch.getSize());
-  }
-
-  public void processRow(CarbonRow row, CarbonFactHandler dataHandler) throws KeyGenException {
-    try {
-      readCounter++;
-      dataHandler.addDataToStore(row);
-    } catch (Exception e) {
-      throw new CarbonDataLoadingException("unable to generate the mdkey", e);
-    }
-    rowCounter.getAndAdd(1);
-  }
-
-  @Override protected CarbonRow processRow(CarbonRow row) {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/InputProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/InputProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/InputProcessorStepImpl.java
deleted file mode 100644
index cbeb20a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/InputProcessorStepImpl.java
+++ /dev/null
@@ -1,244 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.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.atomic.AtomicLong;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.parser.RowParser;
-import org.apache.carbondata.processing.newflow.parser.impl.RowParserImpl;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-
-/**
- * It reads data from record reader and sends data to next step.
- */
-public class InputProcessorStepImpl extends AbstractDataLoadProcessorStep {
-
-  private RowParser rowParser;
-
-  private CarbonIterator<Object[]>[] inputIterators;
-
-  /**
-   * executor service to execute the query
-   */
-  public ExecutorService executorService;
-
-  public InputProcessorStepImpl(CarbonDataLoadConfiguration configuration,
-      CarbonIterator<Object[]>[] inputIterators) {
-    super(configuration, null);
-    this.inputIterators = inputIterators;
-  }
-
-  @Override public DataField[] getOutput() {
-    return configuration.getDataFields();
-  }
-
-  @Override public void initialize() throws IOException {
-    super.initialize();
-    rowParser = new RowParserImpl(getOutput(), configuration);
-    executorService = Executors.newCachedThreadPool();
-  }
-
-  @Override public Iterator<CarbonRowBatch>[] execute() {
-    int batchSize = CarbonProperties.getInstance().getBatchSize();
-    List<CarbonIterator<Object[]>>[] readerIterators = partitionInputReaderIterators();
-    Iterator<CarbonRowBatch>[] outIterators = new Iterator[readerIterators.length];
-    for (int i = 0; i < outIterators.length; i++) {
-      outIterators[i] =
-          new InputProcessorIterator(readerIterators[i], rowParser, batchSize,
-              configuration.isPreFetch(), executorService, rowCounter);
-    }
-    return outIterators;
-  }
-
-  /**
-   * Partition input iterators equally as per the number of threads.
-   * @return
-   */
-  private List<CarbonIterator<Object[]>>[] partitionInputReaderIterators() {
-    // Get the number of cores configured in property.
-    int numberOfCores = CarbonProperties.getInstance().getNumberOfCores();
-    // Get the minimum of number of cores and iterators size to get the number of parallel threads
-    // to be launched.
-    int parallelThreadNumber = Math.min(inputIterators.length, numberOfCores);
-
-    List<CarbonIterator<Object[]>>[] iterators = new List[parallelThreadNumber];
-    for (int i = 0; i < parallelThreadNumber; i++) {
-      iterators[i] = new ArrayList<>();
-    }
-    // Equally partition the iterators as per number of threads
-    for (int i = 0; i < inputIterators.length; i++) {
-      iterators[i % parallelThreadNumber].add(inputIterators[i]);
-    }
-    return iterators;
-  }
-
-  @Override protected CarbonRow processRow(CarbonRow row) {
-    return null;
-  }
-
-  @Override public void close() {
-    if (!closed) {
-      super.close();
-      executorService.shutdown();
-      for (CarbonIterator inputIterator : inputIterators) {
-        inputIterator.close();
-      }
-    }
-  }
-
-  @Override protected String getStepName() {
-    return "Input Processor";
-  }
-
-  /**
-   * This iterator wraps the list of iterators and it starts iterating the each
-   * iterator of the list one by one. It also parse the data while iterating it.
-   */
-  private static class InputProcessorIterator extends CarbonIterator<CarbonRowBatch> {
-
-    private List<CarbonIterator<Object[]>> inputIterators;
-
-    private CarbonIterator<Object[]> currentIterator;
-
-    private int counter;
-
-    private int batchSize;
-
-    private RowParser rowParser;
-
-    private Future<CarbonRowBatch> future;
-
-    private ExecutorService executorService;
-
-    private boolean nextBatch;
-
-    private boolean firstTime;
-
-    private boolean preFetch;
-
-    private AtomicLong rowCounter;
-
-    public InputProcessorIterator(List<CarbonIterator<Object[]>> inputIterators,
-        RowParser rowParser, int batchSize, boolean preFetch, ExecutorService executorService,
-        AtomicLong rowCounter) {
-      this.inputIterators = inputIterators;
-      this.batchSize = batchSize;
-      this.rowParser = rowParser;
-      this.counter = 0;
-      // Get the first iterator from the list.
-      currentIterator = inputIterators.get(counter++);
-      this.executorService = executorService;
-      this.rowCounter = rowCounter;
-      this.preFetch = preFetch;
-      this.nextBatch = false;
-      this.firstTime = true;
-    }
-
-    @Override
-    public boolean hasNext() {
-      return nextBatch || internalHasNext();
-    }
-
-    private boolean internalHasNext() {
-      if (firstTime) {
-        firstTime = false;
-        currentIterator.initialize();
-      }
-      boolean hasNext = currentIterator.hasNext();
-      // If iterator is finished then check for next iterator.
-      if (!hasNext) {
-        currentIterator.close();
-        // Check next iterator is available in the list.
-        if (counter < inputIterators.size()) {
-          // Get the next iterator from the list.
-          currentIterator = inputIterators.get(counter++);
-          // Initialize the new iterator
-          currentIterator.initialize();
-          hasNext = internalHasNext();
-        }
-      }
-      return hasNext;
-    }
-
-    @Override
-    public CarbonRowBatch next() {
-      if (preFetch) {
-        return getCarbonRowBatchWithPreFetch();
-      } else {
-        return getBatch();
-      }
-    }
-
-    private CarbonRowBatch getCarbonRowBatchWithPreFetch() {
-      CarbonRowBatch result = null;
-      if (future == null) {
-        future = getCarbonRowBatch();
-      }
-      try {
-        result = future.get();
-      } catch (InterruptedException e) {
-        throw new RuntimeException(e);
-      } catch (ExecutionException e) {
-        throw new RuntimeException(e);
-      }
-      nextBatch = false;
-      if (hasNext()) {
-        nextBatch = true;
-        future = getCarbonRowBatch();
-      }
-
-      return result;
-    }
-
-    private Future<CarbonRowBatch> getCarbonRowBatch() {
-      return executorService.submit(new Callable<CarbonRowBatch>() {
-        @Override public CarbonRowBatch call() throws Exception {
-          return getBatch();
-
-        }
-      });
-    }
-
-    private CarbonRowBatch getBatch() {
-      // Create batch and fill it.
-      CarbonRowBatch carbonRowBatch = new CarbonRowBatch(batchSize);
-      int count = 0;
-      while (internalHasNext() && count < batchSize) {
-        carbonRowBatch.addRow(new CarbonRow(rowParser.parseRow(currentIterator.next())));
-        count++;
-      }
-      rowCounter.getAndAdd(carbonRowBatch.getSize());
-      return carbonRowBatch;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/SortProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/SortProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/SortProcessorStepImpl.java
deleted file mode 100644
index 0b93b7a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/SortProcessorStepImpl.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.steps;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.sort.Sorter;
-import org.apache.carbondata.processing.newflow.sort.SorterFactory;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-
-/**
- * It sorts the data and write them to intermediate temp files. These files will be further read
- * by next step for writing to carbondata files.
- */
-public class SortProcessorStepImpl extends AbstractDataLoadProcessorStep {
-
-  private Sorter sorter;
-
-  public SortProcessorStepImpl(CarbonDataLoadConfiguration configuration,
-      AbstractDataLoadProcessorStep child) {
-    super(configuration, child);
-  }
-
-  @Override
-  public DataField[] getOutput() {
-    return child.getOutput();
-  }
-
-  @Override
-  public void initialize() throws IOException {
-    super.initialize();
-    child.initialize();
-    SortParameters sortParameters = SortParameters.createSortParameters(configuration);
-    sorter = SorterFactory.createSorter(configuration, rowCounter);
-    sorter.initialize(sortParameters);
-  }
-
-  @Override
-  public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
-    final Iterator<CarbonRowBatch>[] iterators = child.execute();
-    return sorter.sort(iterators);
-  }
-
-  @Override
-  protected CarbonRow processRow(CarbonRow row) {
-    return null;
-  }
-
-  @Override
-  public void close() {
-    if (!closed) {
-      super.close();
-      if (sorter != null) {
-        sorter.close();
-      }
-    }
-  }
-
-  @Override protected String getStepName() {
-    return "Sort Processor";
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java b/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java
new file mode 100644
index 0000000..649c18d
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/DataPartitioner.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.partition;
+
+import java.util.List;
+
+public interface DataPartitioner {
+
+  /**
+   * All the partitions built by the Partitioner
+   */
+  List<Partition> getAllPartitions();
+
+  /**
+   * Identifies the partitions applicable for the given filter (API used for For query)
+   */
+  List<Partition> getPartitions();
+
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java b/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java
new file mode 100644
index 0000000..95cdfb3
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/Partition.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.partition;
+
+import java.io.Serializable;
+import java.util.List;
+
+public interface Partition extends Serializable {
+  /**
+   * unique identification for the partition in the cluster.
+   */
+  String getUniqueID();
+
+  /**
+   * result
+   *
+   * @return
+   */
+  List<String> getFilesPath();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/partition/impl/DefaultLoadBalancer.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/DefaultLoadBalancer.java b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/DefaultLoadBalancer.java
new file mode 100644
index 0000000..e533baf
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/DefaultLoadBalancer.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.partition.impl;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.processing.partition.Partition;
+
+/**
+ * A sample load balancer to distribute the partitions to the available nodes in a round robin mode.
+ */
+public class DefaultLoadBalancer {
+  private Map<String, List<Partition>> nodeToPartitonMap =
+      new HashMap<String, List<Partition>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+  private Map<Partition, String> partitonToNodeMap =
+      new HashMap<Partition, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+  public DefaultLoadBalancer(List<String> nodes, List<Partition> partitions) {
+    //Per form a round robin allocation
+    int nodeCount = nodes.size();
+
+    int partitioner = 0;
+    for (Partition partition : partitions) {
+      int nodeindex = partitioner % nodeCount;
+      String node = nodes.get(nodeindex);
+
+      List<Partition> oldList = nodeToPartitonMap.get(node);
+      if (oldList == null) {
+        oldList = new ArrayList<Partition>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+        nodeToPartitonMap.put(node, oldList);
+      }
+      oldList.add(partition);
+
+      partitonToNodeMap.put(partition, node);
+
+      partitioner++;
+    }
+  }
+
+  public String getNodeForPartitions(Partition partition) {
+    return partitonToNodeMap.get(partition);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/partition/impl/PartitionMultiFileImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/PartitionMultiFileImpl.java b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/PartitionMultiFileImpl.java
new file mode 100644
index 0000000..c303efa
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/PartitionMultiFileImpl.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.partition.impl;
+
+import java.util.List;
+
+import org.apache.carbondata.processing.partition.Partition;
+
+public class PartitionMultiFileImpl implements Partition {
+  private static final long serialVersionUID = -4363447826181193976L;
+  private String uniqueID;
+  private List<String> folderPath;
+
+  public PartitionMultiFileImpl(String uniqueID, List<String> folderPath) {
+    this.uniqueID = uniqueID;
+    this.folderPath = folderPath;
+  }
+
+  @Override public String getUniqueID() {
+    // TODO Auto-generated method stub
+    return uniqueID;
+  }
+
+  @Override public List<String> getFilesPath() {
+    // TODO Auto-generated method stub
+    return folderPath;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..4fb2414
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/QueryPartitionHelper.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.processing.partition.impl;
+
+import java.util.HashMap;
+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.processing.partition.DataPartitioner;
+import org.apache.carbondata.processing.partition.Partition;
+
+
+public final class QueryPartitionHelper {
+  private static QueryPartitionHelper instance = new QueryPartitionHelper();
+  private Map<String, DataPartitioner> partitionerMap =
+      new HashMap<String, DataPartitioner>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  private Map<String, DefaultLoadBalancer> loadBalancerMap =
+      new HashMap<String, DefaultLoadBalancer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+  private QueryPartitionHelper() {
+
+  }
+
+  public static QueryPartitionHelper getInstance() {
+    return instance;
+  }
+
+  /**
+   * Get partitions applicable for query based on filters applied in query
+   */
+  public List<Partition> getPartitionsForQuery(CarbonQueryPlan queryPlan) {
+    String tableUniqueName = queryPlan.getDatabaseName() + '_' + queryPlan.getTableName();
+
+    DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
+
+    return dataPartitioner.getPartitions();
+  }
+
+  public List<Partition> getAllPartitions(String databaseName, String tableName) {
+    String tableUniqueName = databaseName + '_' + tableName;
+
+    DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
+
+    return dataPartitioner.getAllPartitions();
+  }
+
+  /**
+   * Get the node name where the partition is assigned to.
+   */
+  public String getLocation(Partition partition, String databaseName, String tableName) {
+    String tableUniqueName = databaseName + '_' + tableName;
+
+    DefaultLoadBalancer loadBalancer = loadBalancerMap.get(tableUniqueName);
+    return loadBalancer.getNodeForPartitions(partition);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/partition/impl/SampleDataPartitionerImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/impl/SampleDataPartitionerImpl.java b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/SampleDataPartitionerImpl.java
new file mode 100644
index 0000000..92bd6ff
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/impl/SampleDataPartitionerImpl.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.processing.partition.impl;
+
+import java.util.List;
+
+import org.apache.carbondata.processing.partition.DataPartitioner;
+import org.apache.carbondata.processing.partition.Partition;
+
+/**
+ * Sample partition.
+ */
+public class SampleDataPartitionerImpl implements DataPartitioner {
+
+  @Override
+  public List<Partition> getAllPartitions() {
+    return null;
+  }
+
+  @Override
+  public List<Partition> getPartitions() {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..aeddac6
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/AbstractCarbonQueryExecutor.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.processing.partition.spliter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+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.util.CarbonUtil;
+
+public abstract class AbstractCarbonQueryExecutor {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractCarbonQueryExecutor.class.getName());
+  protected CarbonTable carbonTable;
+  protected QueryModel queryModel;
+  protected QueryExecutor queryExecutor;
+  protected Map<String, TaskBlockInfo> segmentMapping;
+
+  /**
+   * get executor and execute the query model.
+   *
+   * @param blockList
+   * @return
+   */
+  protected CarbonIterator<BatchResult> executeBlockList(List<TableBlockInfo> blockList)
+      throws QueryExecutionException, IOException {
+    queryModel.setTableBlockInfos(blockList);
+    this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
+    return queryExecutor.execute(queryModel);
+  }
+
+  /**
+   * 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.getFactTableName());
+    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.getFactTableName());
+    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
+   */
+  public void finish() {
+    try {
+      queryExecutor.finish();
+    } catch (QueryExecutionException e) {
+      LOGGER.error(e, "Problem while finish: ");
+    }
+    clearDictionaryFromQueryModel();
+  }
+
+  /**
+   * This method will clear the dictionary access count after its usage is complete so
+   * that column can be deleted form LRU cache whenever memory reaches threshold
+   */
+  private void clearDictionaryFromQueryModel() {
+    if (null != queryModel) {
+      Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
+      if (null != columnToDictionaryMapping) {
+        for (Map.Entry<String, Dictionary> entry : columnToDictionaryMapping.entrySet()) {
+          CarbonUtil.clearDictionaryCache(entry.getValue());
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..6afec0b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/CarbonSplitExecutor.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.partition.spliter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+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;
+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.scan.executor.exception.QueryExecutionException;
+import org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResultIterator;
+
+/**
+ * Used to read carbon blocks when add/split partition
+ */
+public class CarbonSplitExecutor extends AbstractCarbonQueryExecutor {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CarbonSplitExecutor.class.getName());
+
+  public CarbonSplitExecutor(Map<String, TaskBlockInfo> segmentMapping, CarbonTable carbonTable) {
+    this.segmentMapping = segmentMapping;
+    this.carbonTable = carbonTable;
+  }
+
+  public List<PartitionSpliterRawResultIterator> processDataBlocks(String segmentId)
+      throws QueryExecutionException, IOException {
+    List<TableBlockInfo> list = null;
+    queryModel = prepareQueryModel(list);
+    List<PartitionSpliterRawResultIterator> resultList
+        = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    TaskBlockInfo taskBlockInfo = segmentMapping.get(segmentId);
+    Set<String> taskBlockListMapping = taskBlockInfo.getTaskSet();
+    for (String task : taskBlockListMapping) {
+      list = taskBlockInfo.getTableBlockInfoList(task);
+      LOGGER.info("for task -" + task + "-block size is -" + list.size());
+      queryModel.setTableBlockInfos(list);
+      resultList.add(new PartitionSpliterRawResultIterator(executeBlockList(list)));
+    }
+    return resultList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..1db414f
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/RowResultProcessor.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.partition.spliter;
+
+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.exception.CarbonDataWriterException;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.partition.spliter.exception.AlterPartitionSliceException;
+import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
+import org.apache.carbondata.processing.store.CarbonFactDataHandlerColumnar;
+import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
+import org.apache.carbondata.processing.store.CarbonFactHandler;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+public class RowResultProcessor {
+
+  private CarbonFactHandler dataHandler;
+  private SegmentProperties segmentProperties;
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(RowResultProcessor.class.getName());
+
+
+  public RowResultProcessor(CarbonTable carbonTable, CarbonLoadModel loadModel,
+      SegmentProperties segProp, String[] tempStoreLocation, Integer bucketId) {
+    CarbonDataProcessorUtil.createLocations(tempStoreLocation);
+    this.segmentProperties = segProp;
+    String tableName = carbonTable.getFactTableName();
+    CarbonFactDataHandlerModel carbonFactDataHandlerModel =
+        CarbonFactDataHandlerModel.getCarbonFactDataHandlerModel(loadModel, carbonTable,
+            segProp, tableName, tempStoreLocation);
+    CarbonDataFileAttributes carbonDataFileAttributes =
+        new CarbonDataFileAttributes(Integer.parseInt(loadModel.getTaskNo()),
+            loadModel.getFactTimeStamp());
+    carbonFactDataHandlerModel.setCarbonDataFileAttributes(carbonDataFileAttributes);
+    carbonFactDataHandlerModel.setBucketId(bucketId);
+    //Note: set compaction flow just to convert decimal type
+    carbonFactDataHandlerModel.setCompactionFlow(true);
+    dataHandler = new CarbonFactDataHandlerColumnar(carbonFactDataHandlerModel);
+  }
+
+  public boolean execute(List<Object[]> resultList) {
+    boolean processStatus;
+    boolean isDataPresent = false;
+
+    try {
+      if (!isDataPresent) {
+        dataHandler.initialise();
+        isDataPresent = true;
+      }
+      for (Object[] row: resultList) {
+        addRow(row);
+      }
+      if (isDataPresent)
+      {
+        this.dataHandler.finish();
+      }
+      processStatus = true;
+    } catch (AlterPartitionSliceException e) {
+      LOGGER.error(e, e.getMessage());
+      LOGGER.error("Exception in executing RowResultProcessor" + e.getMessage());
+      processStatus = false;
+    } finally {
+      try {
+        if (isDataPresent) {
+          this.dataHandler.closeHandler();
+        }
+      } catch (Exception e) {
+        LOGGER.error("Exception while closing the handler in RowResultProcessor" + e.getMessage());
+        processStatus = false;
+      }
+    }
+    return processStatus;
+  }
+
+  private void addRow(Object[] carbonTuple) throws AlterPartitionSliceException {
+    CarbonRow row = WriteStepRowUtil.fromMergerRow(carbonTuple, segmentProperties);
+    try {
+      this.dataHandler.addDataToStore(row);
+    } catch (CarbonDataWriterException e) {
+      throw new AlterPartitionSliceException("Exception in adding rows in RowResultProcessor", e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/exception/AlterPartitionSliceException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/exception/AlterPartitionSliceException.java b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/exception/AlterPartitionSliceException.java
new file mode 100644
index 0000000..21b53cf
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/partition/spliter/exception/AlterPartitionSliceException.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.partition.spliter.exception;
+
+import java.util.Locale;
+
+public class AlterPartitionSliceException 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 AlterPartitionSliceException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param msg The error message for this exception.
+   */
+  public AlterPartitionSliceException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * This method is used to get the localized message.
+   *
+   * @param locale - A Locale object represents a specific geographical,
+   *               political, or cultural region.
+   * @return - Localized error message.
+   */
+  public String getLocalizedMessage(Locale locale) {
+    return "";
+  }
+
+  /**
+   * getLocalizedMessage
+   */
+  @Override public String getLocalizedMessage() {
+    return super.getLocalizedMessage();
+  }
+
+  /**
+   * getMessage
+   */
+  public String getMessage() {
+    return this.msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/SortObserver.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/SortObserver.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/SortObserver.java
deleted file mode 100644
index 31c2b4f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/SortObserver.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.schema.metadata;
-
-import java.io.Serializable;
-
-public class SortObserver implements Serializable {
-  /**
-   * is failed
-   */
-  private boolean isFailed;
-
-  /**
-   * @return the isFailed
-   */
-  public boolean isFailed() {
-    return isFailed;
-  }
-
-  /**
-   * @param isFailed the isFailed to set
-   */
-  public void setFailed(boolean isFailed) {
-    this.isFailed = isFailed;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/exception/CarbonSortKeyAndGroupByException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/exception/CarbonSortKeyAndGroupByException.java b/processing/src/main/java/org/apache/carbondata/processing/sort/exception/CarbonSortKeyAndGroupByException.java
new file mode 100644
index 0000000..292cdb3
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/exception/CarbonSortKeyAndGroupByException.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.sort.exception;
+
+import java.util.Locale;
+
+public class CarbonSortKeyAndGroupByException 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 CarbonSortKeyAndGroupByException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param msg The error message for this exception.
+   */
+  public CarbonSortKeyAndGroupByException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param t
+   */
+  public CarbonSortKeyAndGroupByException(Throwable t) {
+    super(t);
+  }
+
+  /**
+   * This method is used to get the localized message.
+   *
+   * @param locale - A Locale object represents a specific geographical,
+   *               political, or cultural region.
+   * @return - Localized error message.
+   */
+  public String getLocalizedMessage(Locale locale) {
+    return "";
+  }
+
+  /**
+   * getLocalizedMessage
+   */
+  @Override public String getLocalizedMessage() {
+    return super.getLocalizedMessage();
+  }
+
+  /**
+   * getMessage
+   */
+  public String getMessage() {
+    return this.msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/AbstractTempSortFileWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/AbstractTempSortFileWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/AbstractTempSortFileWriter.java
new file mode 100644
index 0000000..1302a5b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/AbstractTempSortFileWriter.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.sort.sortdata;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+public abstract class AbstractTempSortFileWriter implements TempSortFileWriter {
+
+  /**
+   * writeFileBufferSize
+   */
+  protected int writeBufferSize;
+
+  /**
+   * Measure count
+   */
+  protected int measureCount;
+
+  /**
+   * Measure count
+   */
+  protected int dimensionCount;
+
+  /**
+   * complexDimension count
+   */
+  protected int complexDimensionCount;
+
+  /**
+   * stream
+   */
+  protected DataOutputStream stream;
+
+  /**
+   * noDictionaryCount
+   */
+  protected int noDictionaryCount;
+
+  /**
+   * AbstractTempSortFileWriter
+   *
+   * @param writeBufferSize
+   * @param dimensionCount
+   * @param measureCount
+   */
+  public AbstractTempSortFileWriter(int dimensionCount, int complexDimensionCount, int measureCount,
+      int noDictionaryCount, int writeBufferSize) {
+    this.writeBufferSize = writeBufferSize;
+    this.dimensionCount = dimensionCount;
+    this.complexDimensionCount = complexDimensionCount;
+    this.measureCount = measureCount;
+    this.noDictionaryCount = noDictionaryCount;
+  }
+
+  /**
+   * Below method will be used to initialize the stream and write the entry count
+   */
+  @Override public void initiaize(File file, int entryCount)
+      throws CarbonSortKeyAndGroupByException {
+    try {
+      stream = new DataOutputStream(
+          new BufferedOutputStream(new FileOutputStream(file), writeBufferSize));
+      stream.writeInt(entryCount);
+    } catch (FileNotFoundException e1) {
+      throw new CarbonSortKeyAndGroupByException(e1);
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException(e);
+    }
+  }
+
+  /**
+   * Below method will be used to close the stream
+   */
+  @Override public void finish() {
+    CarbonUtil.closeStreams(stream);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/CompressedTempSortFileWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/CompressedTempSortFileWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/CompressedTempSortFileWriter.java
new file mode 100644
index 0000000..40f650d
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/CompressedTempSortFileWriter.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.sort.sortdata;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.compression.CompressorFactory;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+public class CompressedTempSortFileWriter extends AbstractTempSortFileWriter {
+
+  /**
+   * CompressedTempSortFileWriter
+   *
+   * @param writeBufferSize
+   * @param dimensionCount
+   * @param measureCount
+   */
+  public CompressedTempSortFileWriter(int dimensionCount, int complexDimensionCount,
+      int measureCount, int noDictionaryCount, int writeBufferSize) {
+    super(dimensionCount, complexDimensionCount, measureCount, noDictionaryCount, writeBufferSize);
+  }
+
+  /**
+   * Below method will be used to write the sort temp file
+   *
+   * @param records
+   */
+  public void writeSortTempFile(Object[][] records) throws CarbonSortKeyAndGroupByException {
+    DataOutputStream dataOutputStream = null;
+    ByteArrayOutputStream blockDataArray = null;
+    int totalSize = 0;
+    int recordSize = 0;
+    try {
+      recordSize = (measureCount * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE) + (dimensionCount
+          * CarbonCommonConstants.INT_SIZE_IN_BYTE);
+      totalSize = records.length * recordSize;
+
+      blockDataArray = new ByteArrayOutputStream(totalSize);
+      dataOutputStream = new DataOutputStream(blockDataArray);
+
+      UnCompressedTempSortFileWriter
+          .writeDataOutputStream(records, dataOutputStream, measureCount, dimensionCount,
+              noDictionaryCount, complexDimensionCount);
+
+      stream.writeInt(records.length);
+      byte[] byteArray = CompressorFactory.getInstance().getCompressor()
+          .compressByte(blockDataArray.toByteArray());
+      stream.writeInt(byteArray.length);
+      stream.write(byteArray);
+
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException(e);
+    } finally {
+      CarbonUtil.closeStreams(blockDataArray);
+      CarbonUtil.closeStreams(dataOutputStream);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..ffe6fb6
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/IntermediateFileMerger.java
@@ -0,0 +1,385 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.AbstractQueue;
+import java.util.PriorityQueue;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+public class IntermediateFileMerger implements Runnable {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(IntermediateFileMerger.class.getName());
+
+  /**
+   * recordHolderHeap
+   */
+  private AbstractQueue<SortTempFileChunkHolder> recordHolderHeap;
+
+  /**
+   * fileCounter
+   */
+  private int fileCounter;
+
+  /**
+   * stream
+   */
+  private DataOutputStream stream;
+
+  /**
+   * totalNumberOfRecords
+   */
+  private int totalNumberOfRecords;
+
+  /**
+   * records
+   */
+  private Object[][] records;
+
+  /**
+   * entryCount
+   */
+  private int entryCount;
+
+  /**
+   * writer
+   */
+  private TempSortFileWriter writer;
+
+  /**
+   * totalSize
+   */
+  private int totalSize;
+
+  private SortParameters mergerParameters;
+
+  private File[] intermediateFiles;
+
+  private File outPutFile;
+
+  private boolean[] noDictionarycolumnMapping;
+
+  /**
+   * IntermediateFileMerger Constructor
+   */
+  public IntermediateFileMerger(SortParameters mergerParameters, File[] intermediateFiles,
+      File outPutFile) {
+    this.mergerParameters = mergerParameters;
+    this.fileCounter = intermediateFiles.length;
+    this.intermediateFiles = intermediateFiles;
+    this.outPutFile = outPutFile;
+    noDictionarycolumnMapping = mergerParameters.getNoDictionaryDimnesionColumn();
+  }
+
+  @Override
+  public void run() {
+    long intermediateMergeStartTime = System.currentTimeMillis();
+    int fileConterConst = fileCounter;
+    boolean isFailed = false;
+    try {
+      startSorting();
+      initialize();
+      while (hasNext()) {
+        writeDataTofile(next());
+      }
+      if (mergerParameters.isSortFileCompressionEnabled() || mergerParameters.isPrefetch()) {
+        if (entryCount > 0) {
+          if (entryCount < totalSize) {
+            Object[][] temp = new Object[entryCount][];
+            System.arraycopy(records, 0, temp, 0, entryCount);
+            records = temp;
+            this.writer.writeSortTempFile(temp);
+          } else {
+            this.writer.writeSortTempFile(records);
+          }
+        }
+      }
+      double intermediateMergeCostTime =
+          (System.currentTimeMillis() - intermediateMergeStartTime) / 1000.0;
+      LOGGER.info("============================== Intermediate Merge of " + fileConterConst +
+          " Sort Temp Files Cost Time: " + intermediateMergeCostTime + "(s)");
+    } catch (Exception e) {
+      LOGGER.error(e, "Problem while intermediate merging");
+      isFailed = true;
+    } finally {
+      records = null;
+      CarbonUtil.closeStreams(this.stream);
+      if (null != writer) {
+        writer.finish();
+      }
+      if (!isFailed) {
+        try {
+          finish();
+        } catch (CarbonSortKeyAndGroupByException e) {
+          LOGGER.error(e, "Problem while deleting the merge file");
+        }
+      } else {
+        if (outPutFile.delete()) {
+          LOGGER.error("Problem while deleting the merge file");
+        }
+      }
+    }
+  }
+
+  /**
+   * This method is responsible for initializing the out stream
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private void initialize() throws CarbonSortKeyAndGroupByException {
+    if (!mergerParameters.isSortFileCompressionEnabled() && !mergerParameters.isPrefetch()) {
+      try {
+        this.stream = new DataOutputStream(
+            new BufferedOutputStream(new FileOutputStream(outPutFile),
+                mergerParameters.getFileWriteBufferSize()));
+        this.stream.writeInt(this.totalNumberOfRecords);
+      } catch (FileNotFoundException e) {
+        throw new CarbonSortKeyAndGroupByException("Problem while getting the file", e);
+      } catch (IOException e) {
+        throw new CarbonSortKeyAndGroupByException("Problem while writing the data to file", e);
+      }
+    } else {
+      writer = TempSortFileWriterFactory.getInstance()
+          .getTempSortFileWriter(mergerParameters.isSortFileCompressionEnabled(),
+              mergerParameters.getDimColCount(), mergerParameters.getComplexDimColCount(),
+              mergerParameters.getMeasureColCount(), mergerParameters.getNoDictionaryCount(),
+              mergerParameters.getFileWriteBufferSize());
+      writer.initiaize(outPutFile, totalNumberOfRecords);
+
+      if (mergerParameters.isPrefetch()) {
+        totalSize = mergerParameters.getBufferSize();
+      } else {
+        totalSize = mergerParameters.getSortTempFileNoOFRecordsInCompression();
+      }
+    }
+  }
+
+  /**
+   * This method will be used to get the sorted record from file
+   *
+   * @return sorted record sorted record
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  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
+    // be based on comparator we are passing the heap
+    // when will call poll it will always delete root of the tree and then
+    // it does trickel down operation complexity is log(n)
+    SortTempFileChunkHolder poll = this.recordHolderHeap.poll();
+
+    // get the row from chunk
+    row = poll.getRow();
+
+    // check if there no entry present
+    if (!poll.hasNext()) {
+      // if chunk is empty then close the stream
+      poll.closeStream();
+
+      // change the file counter
+      --this.fileCounter;
+
+      // reaturn row
+      return row;
+    }
+
+    // read new row
+    poll.readRow();
+
+    // add to heap
+    this.recordHolderHeap.add(poll);
+
+    // return row
+    return row;
+  }
+
+  /**
+   * Below method will be used to start storing process This method will get
+   * all the temp files present in sort temp folder then it will create the
+   * record holder heap and then it will read first record from each file and
+   * initialize the heap
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private void startSorting() throws CarbonSortKeyAndGroupByException {
+    LOGGER.info("Number of temp file: " + this.fileCounter);
+
+    // create record holder heap
+    createRecordHolderQueue(intermediateFiles);
+
+    // iterate over file list and create chunk holder and add to heap
+    LOGGER.info("Started adding first record from each file");
+
+    SortTempFileChunkHolder sortTempFileChunkHolder = null;
+
+    for (File tempFile : intermediateFiles) {
+      // create chunk holder
+      sortTempFileChunkHolder =
+          new SortTempFileChunkHolder(tempFile, mergerParameters.getDimColCount(),
+              mergerParameters.getComplexDimColCount(), mergerParameters.getMeasureColCount(),
+              mergerParameters.getFileBufferSize(), mergerParameters.getNoDictionaryCount(),
+              mergerParameters.getMeasureDataType(),
+              mergerParameters.getNoDictionaryDimnesionColumn(),
+              mergerParameters.getNoDictionarySortColumn());
+
+      // initialize
+      sortTempFileChunkHolder.initialize();
+      sortTempFileChunkHolder.readRow();
+      this.totalNumberOfRecords += sortTempFileChunkHolder.getEntryCount();
+
+      // add to heap
+      this.recordHolderHeap.add(sortTempFileChunkHolder);
+    }
+
+    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
+  }
+
+  /**
+   * This method will be used to create the heap which will be used to hold
+   * the chunk of data
+   *
+   * @param listFiles list of temp files
+   */
+  private void createRecordHolderQueue(File[] listFiles) {
+    // creating record holder heap
+    this.recordHolderHeap = new PriorityQueue<>(listFiles.length);
+  }
+
+  /**
+   * This method will be used to get the sorted row
+   *
+   * @return sorted row
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private Object[] next() throws CarbonSortKeyAndGroupByException {
+    return getSortedRecordFromFile();
+  }
+
+  /**
+   * This method will be used to check whether any more element is present or
+   * not
+   *
+   * @return more element is present
+   */
+  private boolean hasNext() {
+    return this.fileCounter > 0;
+  }
+
+  /**
+   * Below method will be used to write data to file
+   *
+   * @throws CarbonSortKeyAndGroupByException problem while writing
+   */
+  private void writeDataTofile(Object[] row) throws CarbonSortKeyAndGroupByException {
+    if (mergerParameters.isSortFileCompressionEnabled() || mergerParameters.isPrefetch()) {
+      if (entryCount == 0) {
+        records = new Object[totalSize][];
+        records[entryCount++] = row;
+        return;
+      }
+
+      records[entryCount++] = row;
+      if (entryCount == totalSize) {
+        this.writer.writeSortTempFile(records);
+        entryCount = 0;
+        records = new Object[totalSize][];
+      }
+      return;
+    }
+    try {
+      DataType[] aggType = mergerParameters.getMeasureDataType();
+      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 : noDictionarycolumnMapping) {
+        if (nodictinary) {
+          byte[] col = nonDictArray[nonDictKeyIndex++];
+          stream.writeShort(col.length);
+          stream.write(col);
+        } else {
+          stream.writeInt(mdkArray[mdkIndex++]);
+        }
+      }
+
+      int fieldIndex = 0;
+      for (int counter = 0; counter < mergerParameters.getMeasureColCount(); counter++) {
+        if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
+          stream.write((byte) 1);
+          switch (aggType[counter]) {
+            case SHORT:
+              stream.writeShort((short)NonDictionaryUtil.getMeasure(fieldIndex, row));
+              break;
+            case INT:
+              stream.writeInt((int)NonDictionaryUtil.getMeasure(fieldIndex, row));
+              break;
+            case LONG:
+              stream.writeLong((long)NonDictionaryUtil.getMeasure(fieldIndex, row));
+              break;
+            case DOUBLE:
+              stream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
+              break;
+            case DECIMAL:
+              byte[] bigDecimalInBytes = (byte[]) NonDictionaryUtil.getMeasure(fieldIndex, row);
+              stream.writeInt(bigDecimalInBytes.length);
+              stream.write(bigDecimalInBytes);
+              break;
+            default:
+              throw new IllegalArgumentException("unsupported data type:" + aggType[counter]);
+          }
+        } else {
+          stream.write((byte) 0);
+        }
+        fieldIndex++;
+      }
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
+    }
+  }
+
+  private void finish() throws CarbonSortKeyAndGroupByException {
+    if (recordHolderHeap != null) {
+      int size = recordHolderHeap.size();
+      for (int i = 0; i < size; i++) {
+        recordHolderHeap.poll().closeStream();
+      }
+    }
+    try {
+      CarbonUtil.deleteFiles(intermediateFiles);
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException("Problem while deleting the intermediate files");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..d2579d2
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparator.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.sort.sortdata;
+
+import java.util.Comparator;
+
+import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
+
+public class NewRowComparator implements Comparator<Object[]> {
+
+  /**
+   * mapping of dictionary dimensions and no dictionary of sort_column.
+   */
+  private boolean[] noDictionarySortColumnMaping;
+
+  /**
+   * @param noDictionarySortColumnMaping
+   */
+  public NewRowComparator(boolean[] noDictionarySortColumnMaping) {
+    this.noDictionarySortColumnMaping = noDictionarySortColumnMaping;
+  }
+
+  /**
+   * Below method will be used to compare two mdkey
+   */
+  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);
+        if (difference != 0) {
+          return difference;
+        }
+      } else {
+        int dimFieldA = (int) rowA[index];
+        int dimFieldB = (int) rowB[index];
+        diff = dimFieldA - dimFieldB;
+        if (diff != 0) {
+          return diff;
+        }
+      }
+
+      index++;
+    }
+
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..e01b587
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/NewRowComparatorForNormalDims.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.sort.sortdata;
+
+import java.util.Comparator;
+
+/**
+ * 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 NewRowComparatorForNormalDims implements Comparator<Object[]> {
+  /**
+   * dimension count
+   */
+  private int numberOfSortColumns;
+
+  /**
+   * RowComparatorForNormalDims Constructor
+   *
+   * @param numberOfSortColumns
+   */
+  public NewRowComparatorForNormalDims(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 = (int)rowA[i];
+      int dimFieldB = (int)rowB[i];
+      diff = dimFieldA - dimFieldB;
+      if (diff != 0) {
+        return diff;
+      }
+    }
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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;
+  }
+}


[37/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
new file mode 100644
index 0000000..e6db5e2
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
@@ -0,0 +1,326 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.csvinput;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.nio.charset.Charset;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+import com.univocity.parsers.csv.CsvParser;
+import com.univocity.parsers.csv.CsvParserSettings;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.compress.CodecPool;
+import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.io.compress.CompressionCodecFactory;
+import org.apache.hadoop.io.compress.CompressionInputStream;
+import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.io.compress.SplitCompressionInputStream;
+import org.apache.hadoop.io.compress.SplittableCompressionCodec;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.util.LineReader;
+
+/**
+ * An {@link org.apache.hadoop.mapreduce.InputFormat} for csv files.  Files are broken into lines.
+ * Values are the line of csv files.
+ */
+public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWritable> {
+
+  public static final String DELIMITER = "carbon.csvinputformat.delimiter";
+  public static final String DELIMITER_DEFAULT = ",";
+  public static final String COMMENT = "carbon.csvinputformat.comment";
+  public static final String COMMENT_DEFAULT = "#";
+  public static final String QUOTE = "carbon.csvinputformat.quote";
+  public static final String QUOTE_DEFAULT = "\"";
+  public static final String ESCAPE = "carbon.csvinputformat.escape";
+  public static final String ESCAPE_DEFAULT = "\\";
+  public static final String HEADER_PRESENT = "caron.csvinputformat.header.present";
+  public static final boolean HEADER_PRESENT_DEFAULT = false;
+  public static final String READ_BUFFER_SIZE = "carbon.csvinputformat.read.buffer.size";
+  public static final String READ_BUFFER_SIZE_DEFAULT = "65536";
+  public static final String MAX_COLUMNS = "carbon.csvinputformat.max.columns";
+  public static final String NUMBER_OF_COLUMNS = "carbon.csvinputformat.number.of.columns";
+  public static final int DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 2000;
+  public static final int THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 20000;
+
+  private static LogService LOGGER =
+      LogServiceFactory.getLogService(CSVInputFormat.class.toString());
+
+
+  @Override
+  public RecordReader<NullWritable, StringArrayWritable> createRecordReader(InputSplit inputSplit,
+      TaskAttemptContext context) throws IOException, InterruptedException {
+    return new CSVRecordReader();
+  }
+
+  @Override
+  protected boolean isSplitable(JobContext context, Path file) {
+    final CompressionCodec codec = new CompressionCodecFactory(context.getConfiguration())
+        .getCodec(file);
+    if (null == codec) {
+      return true;
+    }
+    return codec instanceof SplittableCompressionCodec;
+  }
+
+  /**
+   * Sets the comment char to configuration. Default it is #.
+   * @param configuration
+   * @param commentChar
+   */
+  public static void setCommentCharacter(Configuration configuration, String commentChar) {
+    if (commentChar != null && !commentChar.isEmpty()) {
+      configuration.set(COMMENT, commentChar);
+    }
+  }
+
+  /**
+   * Sets the delimiter to configuration. Default it is ','
+   * @param configuration
+   * @param delimiter
+   */
+  public static void setCSVDelimiter(Configuration configuration, String delimiter) {
+    if (delimiter != null && !delimiter.isEmpty()) {
+      configuration.set(DELIMITER, delimiter);
+    }
+  }
+
+  /**
+   * Sets the escape character to configuration. Default it is \
+   * @param configuration
+   * @param escapeCharacter
+   */
+  public static void setEscapeCharacter(Configuration configuration, String escapeCharacter) {
+    if (escapeCharacter != null && !escapeCharacter.isEmpty()) {
+      configuration.set(ESCAPE, escapeCharacter);
+    }
+  }
+
+  /**
+   * Whether header needs to read from csv or not. By default it is false.
+   * @param configuration
+   * @param headerExtractEnable
+   */
+  public static void setHeaderExtractionEnabled(Configuration configuration,
+      boolean headerExtractEnable) {
+    configuration.set(HEADER_PRESENT, String.valueOf(headerExtractEnable));
+  }
+
+  /**
+   * Sets the quote character to configuration. Default it is "
+   * @param configuration
+   * @param quoteCharacter
+   */
+  public static void setQuoteCharacter(Configuration configuration, String quoteCharacter) {
+    if (quoteCharacter != null && !quoteCharacter.isEmpty()) {
+      configuration.set(QUOTE, quoteCharacter);
+    }
+  }
+
+  /**
+   * Sets the read buffer size to configuration.
+   * @param configuration
+   * @param bufferSize
+   */
+  public static void setReadBufferSize(Configuration configuration, String bufferSize) {
+    if (bufferSize != null && !bufferSize.isEmpty()) {
+      configuration.set(READ_BUFFER_SIZE, bufferSize);
+    }
+  }
+
+  public static void setMaxColumns(Configuration configuration, String maxColumns) {
+    if (maxColumns != null) {
+      configuration.set(MAX_COLUMNS, maxColumns);
+    }
+  }
+
+  public static void setNumberOfColumns(Configuration configuration, String numberOfColumns) {
+    configuration.set(NUMBER_OF_COLUMNS, numberOfColumns);
+  }
+
+  /**
+   * Treats value as line in file. Key is null.
+   */
+  public static class CSVRecordReader extends RecordReader<NullWritable, StringArrayWritable> {
+
+    private long start;
+    private long end;
+    private BoundedInputStream boundedInputStream;
+    private Reader reader;
+    private CsvParser csvParser;
+    private StringArrayWritable value;
+    private String[] columns;
+    private Seekable filePosition;
+    private boolean isCompressedInput;
+    private Decompressor decompressor;
+
+    @Override
+    public void initialize(InputSplit inputSplit, TaskAttemptContext context)
+        throws IOException, InterruptedException {
+      FileSplit split = (FileSplit) inputSplit;
+      start = split.getStart();
+      end = start + split.getLength();
+      Path file = split.getPath();
+      Configuration job = context.getConfiguration();
+      CompressionCodec codec = (new CompressionCodecFactory(job)).getCodec(file);
+      FileSystem fs = file.getFileSystem(job);
+      int bufferSize = Integer.parseInt(job.get(READ_BUFFER_SIZE, READ_BUFFER_SIZE_DEFAULT));
+      FSDataInputStream fileIn = fs.open(file, bufferSize);
+      InputStream inputStream;
+      if (codec != null) {
+        isCompressedInput = true;
+        decompressor = CodecPool.getDecompressor(codec);
+        if (codec instanceof SplittableCompressionCodec) {
+          SplitCompressionInputStream scIn = ((SplittableCompressionCodec) codec)
+              .createInputStream(fileIn, decompressor, start, end, SplittableCompressionCodec
+                  .READ_MODE.BYBLOCK);
+          start = scIn.getAdjustedStart();
+          end = scIn.getAdjustedEnd();
+          if (start != 0) {
+            LineReader lineReader = new LineReader(scIn, 1);
+            start += lineReader.readLine(new Text(), 0);
+          }
+          filePosition = scIn;
+          inputStream = scIn;
+        } else {
+          CompressionInputStream cIn = codec.createInputStream(fileIn, decompressor);
+          filePosition = cIn;
+          inputStream = cIn;
+        }
+      } else {
+        fileIn.seek(start);
+        if (start != 0) {
+          LineReader lineReader = new LineReader(fileIn, 1);
+          start += lineReader.readLine(new Text(), 0);
+        }
+        boundedInputStream = new BoundedInputStream(fileIn, end - start);
+        filePosition = fileIn;
+        inputStream = boundedInputStream;
+      }
+      reader = new InputStreamReader(inputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      csvParser = new CsvParser(extractCsvParserSettings(job));
+      csvParser.beginParsing(reader);
+    }
+
+    private CsvParserSettings extractCsvParserSettings(Configuration job) {
+      CsvParserSettings parserSettings = new CsvParserSettings();
+      parserSettings.getFormat().setDelimiter(job.get(DELIMITER, DELIMITER_DEFAULT).charAt(0));
+      parserSettings.getFormat().setComment(job.get(COMMENT, COMMENT_DEFAULT).charAt(0));
+      parserSettings.setLineSeparatorDetectionEnabled(true);
+      parserSettings.setNullValue("");
+      parserSettings.setEmptyValue("");
+      parserSettings.setIgnoreLeadingWhitespaces(false);
+      parserSettings.setIgnoreTrailingWhitespaces(false);
+      parserSettings.setSkipEmptyLines(false);
+      parserSettings.setMaxCharsPerColumn(100000);
+      String maxColumns = job.get(MAX_COLUMNS);
+      parserSettings.setMaxColumns(Integer.parseInt(maxColumns));
+      parserSettings.getFormat().setQuote(job.get(QUOTE, QUOTE_DEFAULT).charAt(0));
+      parserSettings.getFormat().setQuoteEscape(job.get(ESCAPE, ESCAPE_DEFAULT).charAt(0));
+      if (start == 0) {
+        parserSettings.setHeaderExtractionEnabled(job.getBoolean(HEADER_PRESENT,
+            HEADER_PRESENT_DEFAULT));
+      }
+      return parserSettings;
+    }
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+      if (csvParser == null) {
+        return false;
+      }
+      columns = csvParser.parseNext();
+      if (columns == null) {
+        value = null;
+        return false;
+      }
+      if (value == null) {
+        value = new StringArrayWritable();
+      }
+      value.set(columns);
+      return true;
+    }
+
+    @Override
+    public NullWritable getCurrentKey() throws IOException, InterruptedException {
+      return NullWritable.get();
+    }
+
+    @Override
+    public StringArrayWritable getCurrentValue() throws IOException, InterruptedException {
+      return value;
+    }
+
+    private long getPos() throws IOException {
+      long retVal = start;
+      if (null != boundedInputStream) {
+        retVal = end - boundedInputStream.getRemaining();
+      } else if (isCompressedInput && null != filePosition) {
+        retVal = filePosition.getPos();
+      }
+      return retVal;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+      return start == end ? 0.0F : Math.min(1.0F, (float) (getPos() -
+          start) / (float) (end - start));
+    }
+
+    @Override
+    public void close() throws IOException {
+      try {
+        if (reader != null) {
+          reader.close();
+        }
+        if (boundedInputStream != null) {
+          boundedInputStream.close();
+        }
+        if (null != csvParser) {
+          csvParser.stopParsing();
+        }
+      } finally {
+        reader = null;
+        boundedInputStream = null;
+        csvParser = null;
+        filePosition = null;
+        value = null;
+        if (decompressor != null) {
+          CodecPool.returnDecompressor(decompressor);
+          decompressor = null;
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVRecordReaderIterator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVRecordReaderIterator.java b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVRecordReaderIterator.java
new file mode 100644
index 0000000..24ef9c1
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVRecordReaderIterator.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.csvinput;
+
+import java.io.IOException;
+
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import com.univocity.parsers.common.TextParsingException;
+
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+/**
+ * It is wrapper iterator around @{@link RecordReader}.
+ */
+public class CSVRecordReaderIterator extends CarbonIterator<Object []> {
+
+  private RecordReader<NullWritable, StringArrayWritable> recordReader;
+
+  /**
+   * It is just a little hack to make recordreader as iterator. Usually we cannot call hasNext
+   * multiple times on record reader as it moves another line. To avoid that situation like hasNext
+   * only tells whether next row is present or not and next will move the pointer to next row after
+   * consuming it.
+   */
+  private boolean isConsumed;
+
+  private InputSplit split;
+
+  private TaskAttemptContext context;
+
+  public CSVRecordReaderIterator(RecordReader<NullWritable, StringArrayWritable> recordReader,
+      InputSplit split, TaskAttemptContext context) {
+    this.recordReader = recordReader;
+    this.split = split;
+    this.context = context;
+  }
+
+  @Override
+  public boolean hasNext() {
+    try {
+      if (!isConsumed) {
+        isConsumed = recordReader.nextKeyValue();
+        return isConsumed;
+      }
+      return true;
+    } catch (Exception e) {
+      if (e instanceof TextParsingException) {
+        throw new CarbonDataLoadingException(
+            CarbonDataProcessorUtil.trimErrorMessage(e.getMessage()));
+      }
+      throw new CarbonDataLoadingException(e);
+    }
+  }
+
+  @Override
+  public Object[] next() {
+    try {
+      String[] data = recordReader.getCurrentValue().get();
+      isConsumed = false;
+      return data;
+    } catch (Exception e) {
+      throw new CarbonDataLoadingException(e);
+    }
+  }
+
+  @Override
+  public void initialize() {
+    try {
+      recordReader.initialize(split, context);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public void close() {
+    try {
+      recordReader.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/StringArrayWritable.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/StringArrayWritable.java b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/StringArrayWritable.java
new file mode 100644
index 0000000..fe4a7ed
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/StringArrayWritable.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.csvinput;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+
+import org.apache.hadoop.io.Writable;
+
+/**
+ * A String sequence that is usable as a key or value.
+ */
+public class StringArrayWritable implements Writable {
+  private String[] values;
+
+  public String[] toStrings() {
+    return values;
+  }
+
+  public void set(String[] values) {
+    this.values = values;
+  }
+
+  public String[] get() {
+    return values;
+  }
+
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    int length = in.readInt();
+    values = new String[length];
+    for (int i = 0; i < length; i++) {
+      byte[] b = new byte[in.readInt()];
+      in.readFully(b);
+      values[i] = new String(b, Charset.defaultCharset());
+    }
+  }
+
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeInt(values.length);                 // write values
+    for (int i = 0; i < values.length; i++) {
+      byte[] b = values[i].getBytes(Charset.defaultCharset());
+      out.writeInt(b.length);
+      out.write(b);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return Arrays.toString(values);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DictionaryServerClientDictionary.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DictionaryServerClientDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DictionaryServerClientDictionary.java
new file mode 100644
index 0000000..25426f6
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DictionaryServerClientDictionary.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.processing.loading.dictionary;
+
+import java.util.Map;
+
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.devapi.BiDictionary;
+import org.apache.carbondata.core.devapi.DictionaryGenerationException;
+import org.apache.carbondata.core.dictionary.client.DictionaryClient;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
+
+/**
+ * Dictionary implementation along with dictionary server client to get new dictionary values
+ */
+public class DictionaryServerClientDictionary implements BiDictionary<Integer, Object> {
+
+  private Dictionary dictionary;
+
+  private DictionaryClient client;
+
+  private Map<Object, Integer> localCache;
+
+  private DictionaryMessage dictionaryMessage;
+
+  private int base;
+
+  public DictionaryServerClientDictionary(Dictionary dictionary, DictionaryClient client,
+      DictionaryMessage key, Map<Object, Integer> localCache) {
+    this.dictionary = dictionary;
+    this.client = client;
+    this.dictionaryMessage = key;
+    this.localCache = localCache;
+    this.base = (dictionary == null ? 0 : dictionary.getDictionaryChunks().getSize() - 1);
+  }
+
+  @Override public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
+    Integer key = getKey(value);
+    if (key == null) {
+      dictionaryMessage.setData(value.toString());
+      DictionaryMessage dictionaryValue = client.getDictionary(dictionaryMessage);
+      key = dictionaryValue.getDictionaryValue();
+      synchronized (localCache) {
+        localCache.put(value, key);
+      }
+      return key + base;
+    }
+    return key;
+  }
+
+  @Override public Integer getKey(Object value) {
+    Integer key = -1;
+    if (dictionary != null) {
+      key = dictionary.getSurrogateKey(value.toString());
+    }
+    if (key == CarbonCommonConstants.INVALID_SURROGATE_KEY) {
+      key = localCache.get(value);
+      if (key != null) {
+        return key + base;
+      }
+    }
+    return key;
+  }
+
+  @Override public Object getValue(Integer key) {
+    throw new UnsupportedOperationException("Not supported here");
+  }
+
+  @Override public int size() {
+    dictionaryMessage.setType(DictionaryMessageType.SIZE);
+    return client.getDictionary(dictionaryMessage).getDictionaryValue() + base;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DirectDictionary.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DirectDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DirectDictionary.java
new file mode 100644
index 0000000..165e5a4
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/DirectDictionary.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.processing.loading.dictionary;
+
+import org.apache.carbondata.core.devapi.BiDictionary;
+import org.apache.carbondata.core.devapi.DictionaryGenerationException;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+
+/**
+ * It is used for generating dictionary from value itself, like timestamp can be used directly as
+ * dictionary.
+ */
+public class DirectDictionary implements BiDictionary<Integer, Object> {
+
+  private DirectDictionaryGenerator dictionaryGenerator;
+
+  public DirectDictionary(DirectDictionaryGenerator dictionaryGenerator) {
+    this.dictionaryGenerator = dictionaryGenerator;
+  }
+
+  @Override
+  public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
+    Integer key = getKey(value);
+    if (key == null) {
+      throw new UnsupportedOperationException("trying to add new entry in DirectDictionary");
+    }
+    return key;
+  }
+
+  @Override
+  public Integer getKey(Object value) {
+    return dictionaryGenerator.generateDirectSurrogateKey(value.toString());
+  }
+
+  @Override
+  public Object getValue(Integer key) {
+    return dictionaryGenerator.getValueFromSurrogate(key);
+  }
+
+  @Override public int size() {
+    return Integer.MAX_VALUE;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/InMemBiDictionary.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/InMemBiDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/InMemBiDictionary.java
new file mode 100644
index 0000000..609fec6
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/InMemBiDictionary.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.dictionary;
+
+import java.util.Map;
+
+import org.apache.carbondata.core.devapi.DictionaryGenerationException;
+import org.apache.carbondata.core.devapi.DictionaryGenerator;
+import org.apache.carbondata.core.devapi.GeneratingBiDictionary;
+
+import com.google.common.collect.BiMap;
+import com.google.common.collect.HashBiMap;
+
+public class InMemBiDictionary<K, V> extends GeneratingBiDictionary<K, V> {
+
+  private BiMap<K, V> biMap;
+
+  /**
+   * Constructor to create a new dictionary, dictionary key will be generated by specified generator
+   * @param generator
+   */
+  public InMemBiDictionary(DictionaryGenerator generator) {
+    super(generator);
+    biMap = HashBiMap.create();
+  }
+
+  /**
+   * Constructor to create a pre-created dictionary
+   * @param preCreatedDictionary
+   */
+  public InMemBiDictionary(Map<K, V> preCreatedDictionary) {
+    super(new DictionaryGenerator<K, V>() {
+      @Override
+      public K generateKey(V value) throws DictionaryGenerationException {
+        // Since dictionary is provided by preCreated, normally it should not come here
+        throw new DictionaryGenerationException(
+            "encounter new dictionary value in pre-created dictionary:" + value);
+      }
+    });
+    biMap = HashBiMap.create(preCreatedDictionary);
+  }
+
+  @Override
+  public K getKey(V value) {
+    return biMap.inverse().get(value);
+  }
+
+  @Override
+  public V getValue(K key) {
+    return biMap.get(key);
+  }
+
+  @Override
+  protected void put(K key, V value) {
+    // dictionary is immutable, it is append only
+    assert (!biMap.containsKey(key));
+    assert (!biMap.containsValue(value));
+    biMap.put(key, value);
+  }
+
+  @Override
+  public int size() {
+    return biMap.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/PreCreatedDictionary.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/PreCreatedDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/PreCreatedDictionary.java
new file mode 100644
index 0000000..b7bd9b7
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/dictionary/PreCreatedDictionary.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.dictionary;
+
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.devapi.BiDictionary;
+import org.apache.carbondata.core.devapi.DictionaryGenerationException;
+
+public class PreCreatedDictionary implements BiDictionary<Integer, Object> {
+
+  private Dictionary dictionary;
+
+  public PreCreatedDictionary(Dictionary dictionary) {
+    this.dictionary = dictionary;
+  }
+
+  @Override
+  public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
+    Integer key = getKey(value);
+    if (key == null) {
+      throw new UnsupportedOperationException("trying to add new entry in PreCreatedDictionary");
+    }
+    return key;
+  }
+
+  @Override
+  public Integer getKey(Object value) {
+    return dictionary.getSurrogateKey(value.toString());
+  }
+
+  @Override
+  public String getValue(Integer key) {
+    return dictionary.getDictionaryValueForKey(key);
+  }
+
+  @Override
+  public int size() {
+    return dictionary.getDictionaryChunks().getSize();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/exception/BadRecordFoundException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/exception/BadRecordFoundException.java b/processing/src/main/java/org/apache/carbondata/processing/loading/exception/BadRecordFoundException.java
new file mode 100644
index 0000000..3c0fe53
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/exception/BadRecordFoundException.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.exception;
+
+public class BadRecordFoundException extends CarbonDataLoadingException {
+  /**
+   * 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 BadRecordFoundException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param msg The error message for this exception.
+   */
+  public BadRecordFoundException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param t
+   */
+  public BadRecordFoundException(Throwable t) {
+    super(t);
+  }
+
+  /**
+   * getMessage
+   */
+  public String getMessage() {
+    return this.msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/exception/CarbonDataLoadingException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/exception/CarbonDataLoadingException.java b/processing/src/main/java/org/apache/carbondata/processing/loading/exception/CarbonDataLoadingException.java
new file mode 100644
index 0000000..9d299a7
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/exception/CarbonDataLoadingException.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.loading.exception;
+
+public class CarbonDataLoadingException extends RuntimeException {
+  /**
+   * 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 CarbonDataLoadingException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param msg The error message for this exception.
+   */
+  public CarbonDataLoadingException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param t
+   */
+  public CarbonDataLoadingException(Throwable t) {
+    super(t);
+  }
+
+  /**
+   * 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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/exception/NoRetryException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/exception/NoRetryException.java b/processing/src/main/java/org/apache/carbondata/processing/loading/exception/NoRetryException.java
new file mode 100644
index 0000000..3533adb
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/exception/NoRetryException.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.exception;
+
+public class NoRetryException extends RuntimeException {
+
+  /**
+   * 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 NoRetryException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param msg The error message for this exception.
+   */
+  public NoRetryException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param t
+   */
+  public NoRetryException(Throwable t) {
+    super(t);
+  }
+
+  /**
+   * getMessage
+   */
+  public String getMessage() {
+    return this.msg;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonDataLoadSchema.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonDataLoadSchema.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonDataLoadSchema.java
new file mode 100644
index 0000000..d7aa103
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonDataLoadSchema.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.processing.loading.model;
+
+import java.io.Serializable;
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+/**
+ * Wrapper Data Load Schema object which will be used to
+ * support relation while data loading
+ */
+public class CarbonDataLoadSchema implements Serializable {
+
+  /**
+   * default serializer
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * CarbonTable info
+   */
+  private CarbonTable carbonTable;
+
+  /**
+   * CarbonDataLoadSchema constructor which takes CarbonTable
+   *
+   * @param carbonTable
+   */
+  public CarbonDataLoadSchema(CarbonTable carbonTable) {
+    this.carbonTable = carbonTable;
+  }
+
+  /**
+   * get carbontable
+   *
+   * @return carbonTable
+   */
+  public CarbonTable getCarbonTable() {
+    return carbonTable;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..39ee270
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
@@ -0,0 +1,764 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Serializable;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
+
+public class CarbonLoadModel implements Serializable {
+
+  private static final long serialVersionUID = 6580168429197697465L;
+
+  private String databaseName;
+
+  private String tableName;
+
+  private String factFilePath;
+
+  private String colDictFilePath;
+
+  private String partitionId;
+
+  private CarbonDataLoadSchema carbonDataLoadSchema;
+
+  private boolean aggLoadRequest;
+
+  private String storePath;
+
+  private boolean isRetentionRequest;
+
+  private String csvHeader;
+  private String[] csvHeaderColumns;
+  private String csvDelimiter;
+  private String complexDelimiterLevel1;
+  private String complexDelimiterLevel2;
+
+  private boolean isDirectLoad;
+  private List<LoadMetadataDetails> loadMetadataDetails;
+  private transient SegmentUpdateStatusManager segmentUpdateStatusManager;
+
+  private String blocksID;
+
+  /**
+   * Map from carbon dimension to pre defined dict file path
+   */
+  private HashMap<CarbonDimension, String> predefDictMap;
+
+  /**
+   * task id, each spark task has a unique id
+   */
+  private String taskNo;
+  /**
+   * new load start time
+   */
+  private long factTimeStamp;
+  /**
+   * load Id
+   */
+  private String segmentId;
+
+  private String allDictPath;
+
+  /**
+   * escape Char
+   */
+  private String escapeChar;
+
+  /**
+   * quote Char
+   */
+  private String quoteChar;
+
+  /**
+   * comment Char
+   */
+  private String commentChar;
+
+  private String dateFormat;
+
+  private String defaultTimestampFormat;
+
+  private String defaultDateFormat;
+
+  /**
+   * defines the string that should be treated as null while loadind data
+   */
+  private String serializationNullFormat;
+
+  /**
+   * defines the string to specify whether the bad record logger should be enabled or not
+   */
+  private String badRecordsLoggerEnable;
+
+  /**
+   * defines the option to specify the bad record logger action
+   */
+  private String badRecordsAction;
+
+  /**
+   * Max number of columns that needs to be parsed by univocity parser
+   */
+  private String maxColumns;
+
+  /**
+   * defines the string to specify whether empty data is good or bad
+   */
+  private String isEmptyDataBadRecord;
+
+  /**
+   * Use one pass to generate dictionary
+   */
+  private boolean useOnePass;
+
+  /**
+   * dictionary server host
+   */
+  private String dictionaryServerHost;
+
+  /**
+   * dictionary sever port
+   */
+  private int dictionaryServerPort;
+
+  /**
+   * Pre fetch data from csv reader
+   */
+  private boolean preFetch;
+
+  /**
+   * Batch sort should be enabled or not
+   */
+  private String sortScope;
+
+  /**
+   * Batch sort size in mb.
+   */
+  private String batchSortSizeInMb;
+  /**
+   * bad record location
+   */
+  private String badRecordsLocation;
+
+  /**
+   * Number of partitions in global sort.
+   */
+  private String globalSortPartitions;
+
+  /**
+   * get escape char
+   *
+   * @return
+   */
+  public String getEscapeChar() {
+    return escapeChar;
+  }
+
+  /**
+   * set escape char
+   *
+   * @param escapeChar
+   */
+  public void setEscapeChar(String escapeChar) {
+    this.escapeChar = escapeChar;
+  }
+
+  public String getCsvDelimiter() {
+    return csvDelimiter;
+  }
+
+  public void setCsvDelimiter(String csvDelimiter) {
+    this.csvDelimiter = csvDelimiter;
+  }
+
+  public String getComplexDelimiterLevel1() {
+    return complexDelimiterLevel1;
+  }
+
+  public void setComplexDelimiterLevel1(String complexDelimiterLevel1) {
+    this.complexDelimiterLevel1 = complexDelimiterLevel1;
+  }
+
+  public String getComplexDelimiterLevel2() {
+    return complexDelimiterLevel2;
+  }
+
+  public void setComplexDelimiterLevel2(String complexDelimiterLevel2) {
+    this.complexDelimiterLevel2 = complexDelimiterLevel2;
+  }
+
+  public boolean isDirectLoad() {
+    return isDirectLoad;
+  }
+
+  public void setDirectLoad(boolean isDirectLoad) {
+    this.isDirectLoad = isDirectLoad;
+  }
+
+  public String getAllDictPath() {
+    return allDictPath;
+  }
+
+  public void setAllDictPath(String allDictPath) {
+    this.allDictPath = allDictPath;
+  }
+
+  public String getCsvHeader() {
+    return csvHeader;
+  }
+
+  public void setCsvHeader(String csvHeader) {
+    this.csvHeader = csvHeader;
+  }
+
+  public String[] getCsvHeaderColumns() {
+    return csvHeaderColumns;
+  }
+
+  public void setCsvHeaderColumns(String[] csvHeaderColumns) {
+    this.csvHeaderColumns = csvHeaderColumns;
+  }
+
+  public void initPredefDictMap() {
+    predefDictMap = new HashMap<>();
+  }
+
+  public String getPredefDictFilePath(CarbonDimension dimension) {
+    return predefDictMap.get(dimension);
+  }
+
+  public void setPredefDictMap(CarbonDimension dimension, String predefDictFilePath) {
+    this.predefDictMap.put(dimension, predefDictFilePath);
+  }
+
+  /**
+   * @return carbon dataload schema
+   */
+  public CarbonDataLoadSchema getCarbonDataLoadSchema() {
+    return carbonDataLoadSchema;
+  }
+
+  /**
+   * @param carbonDataLoadSchema
+   */
+  public void setCarbonDataLoadSchema(CarbonDataLoadSchema carbonDataLoadSchema) {
+    this.carbonDataLoadSchema = carbonDataLoadSchema;
+  }
+
+  /**
+   * @return the databaseName
+   */
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  /**
+   * @param databaseName the databaseName to set
+   */
+  public void setDatabaseName(String databaseName) {
+    this.databaseName = databaseName;
+  }
+
+  /**
+   * @return the tableName
+   */
+  public String getTableName() {
+    return tableName;
+  }
+
+  /**
+   * @param tableName the tableName to set
+   */
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  /**
+   * @return the factFilePath
+   */
+  public String getFactFilePath() {
+    return factFilePath;
+  }
+
+  /**
+   * @param factFilePath the factFilePath to set
+   */
+  public void setFactFilePath(String factFilePath) {
+    this.factFilePath = factFilePath;
+  }
+
+  /**
+   * @return external column dictionary file path
+   */
+  public String getColDictFilePath() {
+    return colDictFilePath;
+  }
+
+  /**
+   * set external column dictionary file path
+   *
+   * @param colDictFilePath
+   */
+  public void setColDictFilePath(String colDictFilePath) {
+    this.colDictFilePath = colDictFilePath;
+  }
+
+  /**
+   * get copy with partition
+   *
+   * @param uniqueId
+   * @return
+   */
+  public CarbonLoadModel getCopyWithPartition(String uniqueId) {
+    CarbonLoadModel copy = new CarbonLoadModel();
+    copy.tableName = tableName;
+    copy.factFilePath = factFilePath + '/' + uniqueId;
+    copy.databaseName = databaseName;
+    copy.partitionId = uniqueId;
+    copy.aggLoadRequest = aggLoadRequest;
+    copy.loadMetadataDetails = loadMetadataDetails;
+    copy.isRetentionRequest = isRetentionRequest;
+    copy.complexDelimiterLevel1 = complexDelimiterLevel1;
+    copy.complexDelimiterLevel2 = complexDelimiterLevel2;
+    copy.carbonDataLoadSchema = carbonDataLoadSchema;
+    copy.blocksID = blocksID;
+    copy.taskNo = taskNo;
+    copy.factTimeStamp = factTimeStamp;
+    copy.segmentId = segmentId;
+    copy.serializationNullFormat = serializationNullFormat;
+    copy.badRecordsLoggerEnable = badRecordsLoggerEnable;
+    copy.badRecordsAction = badRecordsAction;
+    copy.escapeChar = escapeChar;
+    copy.quoteChar = quoteChar;
+    copy.commentChar = commentChar;
+    copy.dateFormat = dateFormat;
+    copy.defaultTimestampFormat = defaultTimestampFormat;
+    copy.maxColumns = maxColumns;
+    copy.storePath = storePath;
+    copy.useOnePass = useOnePass;
+    copy.dictionaryServerHost = dictionaryServerHost;
+    copy.dictionaryServerPort = dictionaryServerPort;
+    copy.preFetch = preFetch;
+    copy.isEmptyDataBadRecord = isEmptyDataBadRecord;
+    copy.sortScope = sortScope;
+    copy.batchSortSizeInMb = batchSortSizeInMb;
+    copy.badRecordsLocation = badRecordsLocation;
+    return copy;
+  }
+
+  /**
+   * Get copy with taskNo.
+   * Broadcast value is shared in process, so we need to copy it to make sure the value in each
+   * task independently.
+   *
+   * @return
+   */
+  public CarbonLoadModel getCopyWithTaskNo(String taskNo) {
+    CarbonLoadModel copy = new CarbonLoadModel();
+    copy.tableName = tableName;
+    copy.factFilePath = factFilePath;
+    copy.databaseName = databaseName;
+    copy.partitionId = partitionId;
+    copy.aggLoadRequest = aggLoadRequest;
+    copy.loadMetadataDetails = loadMetadataDetails;
+    copy.isRetentionRequest = isRetentionRequest;
+    copy.csvHeader = csvHeader;
+    copy.csvHeaderColumns = csvHeaderColumns;
+    copy.isDirectLoad = isDirectLoad;
+    copy.csvDelimiter = csvDelimiter;
+    copy.complexDelimiterLevel1 = complexDelimiterLevel1;
+    copy.complexDelimiterLevel2 = complexDelimiterLevel2;
+    copy.carbonDataLoadSchema = carbonDataLoadSchema;
+    copy.blocksID = blocksID;
+    copy.taskNo = taskNo;
+    copy.factTimeStamp = factTimeStamp;
+    copy.segmentId = segmentId;
+    copy.serializationNullFormat = serializationNullFormat;
+    copy.badRecordsLoggerEnable = badRecordsLoggerEnable;
+    copy.badRecordsAction = badRecordsAction;
+    copy.escapeChar = escapeChar;
+    copy.quoteChar = quoteChar;
+    copy.commentChar = commentChar;
+    copy.dateFormat = dateFormat;
+    copy.defaultTimestampFormat = defaultTimestampFormat;
+    copy.maxColumns = maxColumns;
+    copy.storePath = storePath;
+    copy.useOnePass = useOnePass;
+    copy.dictionaryServerHost = dictionaryServerHost;
+    copy.dictionaryServerPort = dictionaryServerPort;
+    copy.preFetch = preFetch;
+    copy.isEmptyDataBadRecord = isEmptyDataBadRecord;
+    copy.sortScope = sortScope;
+    copy.batchSortSizeInMb = batchSortSizeInMb;
+    return copy;
+  }
+
+  /**
+   * get CarbonLoadModel with partition
+   *
+   * @param uniqueId
+   * @param filesForPartition
+   * @param header
+   * @param delimiter
+   * @return
+   */
+  public CarbonLoadModel getCopyWithPartition(String uniqueId, List<String> filesForPartition,
+      String header, String delimiter) {
+    CarbonLoadModel copyObj = new CarbonLoadModel();
+    copyObj.tableName = tableName;
+    copyObj.factFilePath = null;
+    copyObj.databaseName = databaseName;
+    copyObj.partitionId = uniqueId;
+    copyObj.aggLoadRequest = aggLoadRequest;
+    copyObj.loadMetadataDetails = loadMetadataDetails;
+    copyObj.isRetentionRequest = isRetentionRequest;
+    copyObj.carbonDataLoadSchema = carbonDataLoadSchema;
+    copyObj.csvHeader = header;
+    copyObj.csvHeaderColumns = csvHeaderColumns;
+    copyObj.isDirectLoad = true;
+    copyObj.csvDelimiter = delimiter;
+    copyObj.complexDelimiterLevel1 = complexDelimiterLevel1;
+    copyObj.complexDelimiterLevel2 = complexDelimiterLevel2;
+    copyObj.blocksID = blocksID;
+    copyObj.taskNo = taskNo;
+    copyObj.factTimeStamp = factTimeStamp;
+    copyObj.segmentId = segmentId;
+    copyObj.serializationNullFormat = serializationNullFormat;
+    copyObj.badRecordsLoggerEnable = badRecordsLoggerEnable;
+    copyObj.badRecordsAction = badRecordsAction;
+    copyObj.escapeChar = escapeChar;
+    copyObj.quoteChar = quoteChar;
+    copyObj.commentChar = commentChar;
+    copyObj.dateFormat = dateFormat;
+    copyObj.defaultTimestampFormat = defaultTimestampFormat;
+    copyObj.maxColumns = maxColumns;
+    copyObj.storePath = storePath;
+    copyObj.useOnePass = useOnePass;
+    copyObj.dictionaryServerHost = dictionaryServerHost;
+    copyObj.dictionaryServerPort = dictionaryServerPort;
+    copyObj.preFetch = preFetch;
+    copyObj.isEmptyDataBadRecord = isEmptyDataBadRecord;
+    copyObj.sortScope = sortScope;
+    copyObj.batchSortSizeInMb = batchSortSizeInMb;
+    copyObj.badRecordsLocation = badRecordsLocation;
+    return copyObj;
+  }
+
+  /**
+   * @return the partitionId
+   */
+  public String getPartitionId() {
+    return partitionId;
+  }
+
+  /**
+   * @param partitionId the partitionId to set
+   */
+  public void setPartitionId(String partitionId) {
+    this.partitionId = partitionId;
+  }
+
+  /**
+   * @param storePath The storePath to set.
+   */
+  public void setStorePath(String storePath) {
+    this.storePath = storePath;
+  }
+
+  /**
+   * @return Returns the factStoreLocation.
+   */
+  public String getStorePath() {
+    return storePath;
+  }
+
+  /**
+   * isRetentionRequest
+   *
+   * @return
+   */
+  public boolean isRetentionRequest() {
+    return isRetentionRequest;
+  }
+
+  /**
+   * getLoadMetadataDetails.
+   *
+   * @return
+   */
+  public List<LoadMetadataDetails> getLoadMetadataDetails() {
+    return loadMetadataDetails;
+  }
+
+  /**
+   * setLoadMetadataDetails.
+   *
+   * @param loadMetadataDetails
+   */
+  public void setLoadMetadataDetails(List<LoadMetadataDetails> loadMetadataDetails) {
+    this.loadMetadataDetails = loadMetadataDetails;
+  }
+
+  /**
+   * getSegmentUpdateStatusManager
+   *
+   * @return
+   */
+  public SegmentUpdateStatusManager getSegmentUpdateStatusManager() {
+    return segmentUpdateStatusManager;
+  }
+
+  /**
+   * setSegmentUpdateStatusManager
+   *
+   * @param segmentUpdateStatusManager
+   */
+  public void setSegmentUpdateStatusManager(SegmentUpdateStatusManager segmentUpdateStatusManager) {
+    this.segmentUpdateStatusManager = segmentUpdateStatusManager;
+  }
+
+  /**
+   * @return
+   */
+  public String getTaskNo() {
+    return taskNo;
+  }
+
+  /**
+   * @param taskNo
+   */
+  public void setTaskNo(String taskNo) {
+    this.taskNo = taskNo;
+  }
+
+  /**
+   * @return
+   */
+  public long getFactTimeStamp() {
+    return factTimeStamp;
+  }
+
+  /**
+   * @param factTimeStamp
+   */
+  public void setFactTimeStamp(long factTimeStamp) {
+    this.factTimeStamp = factTimeStamp;
+  }
+
+  public String[] getDelimiters() {
+    return new String[] { complexDelimiterLevel1, complexDelimiterLevel2 };
+  }
+
+  /**
+   * @return load Id
+   */
+  public String getSegmentId() {
+    return segmentId;
+  }
+
+  /**
+   * @param segmentId
+   */
+  public void setSegmentId(String segmentId) {
+    this.segmentId = segmentId;
+  }
+
+  /**
+   * the method returns the value to be treated as null while data load
+   *
+   * @return
+   */
+  public String getSerializationNullFormat() {
+    return serializationNullFormat;
+  }
+
+  /**
+   * the method sets the value to be treated as null while data load
+   *
+   * @param serializationNullFormat
+   */
+  public void setSerializationNullFormat(String serializationNullFormat) {
+    this.serializationNullFormat = serializationNullFormat;
+  }
+
+  /**
+   * returns the string to enable bad record logger
+   *
+   * @return
+   */
+  public String getBadRecordsLoggerEnable() {
+    return badRecordsLoggerEnable;
+  }
+
+  /**
+   * method sets the string to specify whether to enable or dissable the badrecord logger.
+   *
+   * @param badRecordsLoggerEnable
+   */
+  public void setBadRecordsLoggerEnable(String badRecordsLoggerEnable) {
+    this.badRecordsLoggerEnable = badRecordsLoggerEnable;
+  }
+
+  public String getQuoteChar() {
+    return quoteChar;
+  }
+
+  public void setQuoteChar(String quoteChar) {
+    this.quoteChar = quoteChar;
+  }
+
+  public String getCommentChar() {
+    return commentChar;
+  }
+
+  public void setCommentChar(String commentChar) {
+    this.commentChar = commentChar;
+  }
+
+  public String getDateFormat() {
+    return dateFormat;
+  }
+
+  public void setDateFormat(String dateFormat) {
+    this.dateFormat = dateFormat;
+  }
+
+  public String getDefaultTimestampFormat() {
+    return defaultTimestampFormat;
+  }
+
+  public void setDefaultTimestampFormat(String defaultTimestampFormat) {
+    this.defaultTimestampFormat = defaultTimestampFormat;
+  }
+
+  /**
+   * @return
+   */
+  public String getMaxColumns() {
+    return maxColumns;
+  }
+
+  /**
+   * @param maxColumns
+   */
+  public void setMaxColumns(String maxColumns) {
+    this.maxColumns = maxColumns;
+  }
+
+  /**
+   * returns option to specify the bad record logger action
+   *
+   * @return
+   */
+  public String getBadRecordsAction() {
+    return badRecordsAction;
+  }
+
+  /**
+   * set option to specify the bad record logger action
+   *
+   * @param badRecordsAction
+   */
+  public void setBadRecordsAction(String badRecordsAction) {
+    this.badRecordsAction = badRecordsAction;
+  }
+
+  public boolean getUseOnePass() {
+    return useOnePass;
+  }
+
+  public void setUseOnePass(boolean useOnePass) {
+    this.useOnePass = useOnePass;
+  }
+
+  public int getDictionaryServerPort() {
+    return dictionaryServerPort;
+  }
+
+  public void setDictionaryServerPort(int dictionaryServerPort) {
+    this.dictionaryServerPort = dictionaryServerPort;
+  }
+
+  public String getDictionaryServerHost() {
+    return dictionaryServerHost;
+  }
+
+  public void setDictionaryServerHost(String dictionaryServerHost) {
+    this.dictionaryServerHost = dictionaryServerHost;
+  }
+
+  public boolean isPreFetch() {
+    return preFetch;
+  }
+
+  public void setPreFetch(boolean preFetch) {
+    this.preFetch = preFetch;
+  }
+
+  public String getDefaultDateFormat() {
+    return defaultDateFormat;
+  }
+
+  public void setDefaultDateFormat(String defaultDateFormat) {
+    this.defaultDateFormat = defaultDateFormat;
+  }
+
+  public String getIsEmptyDataBadRecord() {
+    return isEmptyDataBadRecord;
+  }
+
+  public void setIsEmptyDataBadRecord(String isEmptyDataBadRecord) {
+    this.isEmptyDataBadRecord = isEmptyDataBadRecord;
+  }
+
+  public String getSortScope() {
+    return sortScope;
+  }
+
+  public void setSortScope(String sortScope) {
+    this.sortScope = sortScope;
+  }
+
+  public String getBatchSortSizeInMb() {
+    return batchSortSizeInMb;
+  }
+
+  public void setBatchSortSizeInMb(String batchSortSizeInMb) {
+    this.batchSortSizeInMb = batchSortSizeInMb;
+  }
+
+  public String getGlobalSortPartitions() {
+    return globalSortPartitions;
+  }
+
+  public void setGlobalSortPartitions(String globalSortPartitions) {
+    this.globalSortPartitions = globalSortPartitions;
+  }
+
+  public String getBadRecordsLocation() {
+    return badRecordsLocation;
+  }
+
+  public void setBadRecordsLocation(String badRecordsLocation) {
+    this.badRecordsLocation = badRecordsLocation;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
new file mode 100644
index 0000000..0ee1d90
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/CarbonParserFactory.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.parser;
+
+import java.util.List;
+
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.processing.loading.parser.impl.ArrayParserImpl;
+import org.apache.carbondata.processing.loading.parser.impl.PrimitiveParserImpl;
+import org.apache.carbondata.processing.loading.parser.impl.StructParserImpl;
+
+public final class CarbonParserFactory {
+
+  /**
+   * Create parser for the carbon column.
+   *
+   * @param carbonColumn
+   * @param complexDelimiters
+   * @return
+   */
+  public static GenericParser createParser(CarbonColumn carbonColumn, String[] complexDelimiters,
+      String nullFormat) {
+    return createParser(carbonColumn, complexDelimiters, nullFormat, 0);
+  }
+
+  /**
+   * This method may be called recursively if the carbon column is complex type.
+   *
+   * @param carbonColumn
+   * @param complexDelimiters, these delimiters which are used to separate the complex data types.
+   * @param depth              It is like depth of tree, if column has children then depth is 1,
+   *                           And depth becomes 2 if children has children.
+   *                           This depth is used select the complex
+   *                           delimiters
+   * @return GenericParser
+   */
+  private static GenericParser createParser(CarbonColumn carbonColumn, String[] complexDelimiters,
+      String nullFormat, int depth) {
+    switch (carbonColumn.getDataType()) {
+      case ARRAY:
+        List<CarbonDimension> listOfChildDimensions =
+            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
+        // Create array parser with complex delimiter
+        ArrayParserImpl arrayParser = new ArrayParserImpl(complexDelimiters[depth], nullFormat);
+        for (CarbonDimension dimension : listOfChildDimensions) {
+          arrayParser
+              .addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
+        }
+        return arrayParser;
+      case STRUCT:
+        List<CarbonDimension> dimensions =
+            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
+        // Create struct parser with complex delimiter
+        StructParserImpl parser = new StructParserImpl(complexDelimiters[depth], nullFormat);
+        for (CarbonDimension dimension : dimensions) {
+          parser.addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
+        }
+        return parser;
+      case MAP:
+        throw new UnsupportedOperationException("Complex type Map is not supported yet");
+      default:
+        return new PrimitiveParserImpl();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/parser/ComplexParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/ComplexParser.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/ComplexParser.java
new file mode 100644
index 0000000..c20766a
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/ComplexParser.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.processing.loading.parser;
+
+/**
+ * It parses data string as per complex data type.
+ */
+public interface ComplexParser<E> extends GenericParser<E> {
+
+  /**
+   * Children to this parser.
+   * @param parser
+   */
+  void addChildren(GenericParser parser);
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/parser/GenericParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/GenericParser.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/GenericParser.java
new file mode 100644
index 0000000..792c409
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/GenericParser.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.processing.loading.parser;
+
+/**
+ * Parse the data according to implementation, The implementation classes can be struct, array or
+ * map datatypes.
+ * It remains thread safe as the state of implementation class should not change while
+ * calling @{@link GenericParser#parse(Object)} method
+ */
+public interface GenericParser<E> {
+
+  /**
+   * Parse the data as per the delimiter
+   * @param data
+   * @return
+   */
+  E parse(Object data);
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/parser/RowParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/RowParser.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/RowParser.java
new file mode 100644
index 0000000..9a74a41
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/RowParser.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.parser;
+
+/**
+ * Parse the complete row at once.
+ */
+public interface RowParser {
+
+  /**
+   * Parse row.
+   * @param row input row to be parsed.
+   * @return parsed row.
+   */
+  Object[] parseRow(Object[] row);
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/ArrayParserImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/ArrayParserImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/ArrayParserImpl.java
new file mode 100644
index 0000000..c56691a
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/ArrayParserImpl.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.parser.impl;
+
+import java.util.regex.Pattern;
+
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.processing.loading.complexobjects.ArrayObject;
+import org.apache.carbondata.processing.loading.parser.ComplexParser;
+import org.apache.carbondata.processing.loading.parser.GenericParser;
+
+import org.apache.commons.lang.ArrayUtils;
+
+/**
+ * It parses the string to @{@link ArrayObject} using delimiter.
+ * It is thread safe as the state of class don't change while
+ * calling @{@link GenericParser#parse(Object)} method
+ */
+public class ArrayParserImpl implements ComplexParser<ArrayObject> {
+
+  private Pattern pattern;
+
+  private GenericParser child;
+
+  private String nullFormat;
+
+  public ArrayParserImpl(String delimiter, String nullFormat) {
+    pattern = Pattern.compile(CarbonUtil.delimiterConverter(delimiter));
+    this.nullFormat = nullFormat;
+  }
+
+  @Override
+  public ArrayObject parse(Object data) {
+    if (data != null) {
+      String value = data.toString();
+      if (!value.isEmpty() && !value.equals(nullFormat)) {
+        String[] split = pattern.split(value, -1);
+        if (ArrayUtils.isNotEmpty(split)) {
+          Object[] array = new Object[split.length];
+          for (int i = 0; i < split.length; i++) {
+            array[i] = child.parse(split[i]);
+          }
+          return new ArrayObject(array);
+        }
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public void addChildren(GenericParser parser) {
+    child = parser;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/RowParserImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/RowParserImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/RowParserImpl.java
new file mode 100644
index 0000000..6f7c398
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/RowParserImpl.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.parser.impl;
+
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.loading.parser.CarbonParserFactory;
+import org.apache.carbondata.processing.loading.parser.GenericParser;
+import org.apache.carbondata.processing.loading.parser.RowParser;
+
+public class RowParserImpl implements RowParser {
+
+  private GenericParser[] genericParsers;
+
+  private int[] outputMapping;
+
+  private int[] inputMapping;
+
+  private int numberOfColumns;
+
+  public RowParserImpl(DataField[] output, CarbonDataLoadConfiguration configuration) {
+    String[] complexDelimiters =
+        (String[]) configuration.getDataLoadProperty(DataLoadProcessorConstants.COMPLEX_DELIMITERS);
+    String nullFormat =
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
+            .toString();
+    DataField[] input = getInput(configuration);
+    genericParsers = new GenericParser[input.length];
+    for (int i = 0; i < genericParsers.length; i++) {
+      genericParsers[i] =
+          CarbonParserFactory.createParser(input[i].getColumn(), complexDelimiters, nullFormat);
+    }
+    outputMapping = new int[output.length];
+    for (int i = 0; i < input.length; i++) {
+      for (int j = 0; j < output.length; j++) {
+        if (input[i].getColumn().equals(output[j].getColumn())) {
+          outputMapping[i] = j;
+          break;
+        }
+      }
+    }
+  }
+
+  public DataField[] getInput(CarbonDataLoadConfiguration configuration) {
+    DataField[] fields = configuration.getDataFields();
+    String[] header = configuration.getHeader();
+    numberOfColumns = header.length;
+    DataField[] input = new DataField[fields.length];
+    inputMapping = new int[input.length];
+    int k = 0;
+    for (int i = 0; i < fields.length; i++) {
+      for (int j = 0; j < numberOfColumns; j++) {
+        if (header[j].equalsIgnoreCase(fields[i].getColumn().getColName())) {
+          input[k] = fields[i];
+          inputMapping[k] = j;
+          k++;
+          break;
+        }
+      }
+    }
+    return input;
+  }
+
+  @Override
+  public Object[] parseRow(Object[] row) {
+    // If number of columns are less in a row then create new array with same size of header.
+    if (row.length < numberOfColumns) {
+      String[] temp = new String[numberOfColumns];
+      System.arraycopy(row, 0, temp, 0, row.length);
+      row = temp;
+    }
+    Object[] out = new Object[genericParsers.length];
+    for (int i = 0; i < genericParsers.length; i++) {
+      Object obj = row[inputMapping[i]];
+      out[outputMapping[i]] = genericParsers[i].parse(obj);
+    }
+    return out;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/StructParserImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/StructParserImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/StructParserImpl.java
new file mode 100644
index 0000000..26744b0
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/StructParserImpl.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.parser.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.processing.loading.complexobjects.StructObject;
+import org.apache.carbondata.processing.loading.parser.ComplexParser;
+import org.apache.carbondata.processing.loading.parser.GenericParser;
+
+import org.apache.commons.lang.ArrayUtils;
+
+/**
+ * It parses the string to @{@link StructObject} using delimiter.
+ * It is thread safe as the state of class don't change while
+ * calling @{@link GenericParser#parse(Object)} method
+ */
+public class StructParserImpl implements ComplexParser<StructObject> {
+
+  private Pattern pattern;
+
+  private List<GenericParser> children = new ArrayList<>();
+
+  private String nullFormat;
+
+  public StructParserImpl(String delimiter, String nullFormat) {
+    pattern = Pattern.compile(CarbonUtil.delimiterConverter(delimiter));
+    this.nullFormat = nullFormat;
+  }
+
+  @Override
+  public StructObject parse(Object data) {
+    if (data != null) {
+      String value = data.toString();
+      if (!value.isEmpty() && !value.equals(nullFormat)) {
+        String[] split = pattern.split(value, -1);
+        if (ArrayUtils.isNotEmpty(split)) {
+          Object[] array = new Object[children.size()];
+          for (int i = 0; i < split.length && i < array.length; i++) {
+            array[i] = children.get(i).parse(split[i]);
+          }
+          return new StructObject(array);
+        }
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public void addChildren(GenericParser parser) {
+    children.add(parser);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/partition/Partitioner.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/partition/Partitioner.java b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/Partitioner.java
new file mode 100644
index 0000000..f71d407
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/Partitioner.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+/**
+ * Partitions the data as per key
+ */
+public interface Partitioner<Key> {
+
+  int getPartition(Key key);
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..06bd716
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/HashPartitionerImpl.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.List;
+
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.processing.loading.partition.Partitioner;
+
+/**
+ * Hash partitioner implementation
+ */
+public class HashPartitionerImpl implements Partitioner<Object[]> {
+
+  private int numberOfBuckets;
+
+  private Hash[] hashes;
+
+  public HashPartitionerImpl(List<Integer> indexes, List<ColumnSchema> columnSchemas,
+      int numberOfBuckets) {
+    this.numberOfBuckets = numberOfBuckets;
+    hashes = new Hash[indexes.size()];
+    for (int i = 0; i < indexes.size(); i++) {
+      switch (columnSchemas.get(i).getDataType()) {
+        case SHORT:
+        case INT:
+        case LONG:
+          hashes[i] = new IntegralHash(indexes.get(i));
+          break;
+        case DOUBLE:
+        case FLOAT:
+        case DECIMAL:
+          hashes[i] = new DecimalHash(indexes.get(i));
+          break;
+        default:
+          hashes[i] = new StringHash(indexes.get(i));
+      }
+    }
+  }
+
+  @Override public int getPartition(Object[] objects) {
+    int hashCode = 0;
+    for (Hash hash : hashes) {
+      hashCode += hash.getHash(objects);
+    }
+    return (hashCode & Integer.MAX_VALUE) % numberOfBuckets;
+  }
+
+  private interface Hash {
+    int getHash(Object[] value);
+  }
+
+  private static class IntegralHash implements Hash {
+
+    private int index;
+
+    private IntegralHash(int index) {
+      this.index = index;
+    }
+
+    public int getHash(Object[] value) {
+      return value[index] != null ? Long.valueOf(value[index].toString()).hashCode() : 0;
+    }
+  }
+
+  private static class DecimalHash implements Hash {
+
+    private int index;
+
+    private DecimalHash(int index) {
+      this.index = index;
+    }
+
+    public int getHash(Object[] value) {
+      return value[index] != null ? Double.valueOf(value[index].toString()).hashCode() : 0;
+    }
+  }
+
+  private static class StringHash implements Hash {
+
+    private int index;
+
+    private StringHash(int index) {
+      this.index = index;
+    }
+
+    @Override public int getHash(Object[] value) {
+      return value[index] != null ? value[index].hashCode() : 0;
+    }
+  }
+}


[05/50] [abbrv] carbondata git commit: [CARBONDATA-1449]Fixed date and timestamp filter gc issue in case of direct dictionary

Posted by ja...@apache.org.
[CARBONDATA-1449]Fixed date and timestamp filter gc issue in case of direct dictionary

Problem: When date or timestamp filter is going to row level there is lots of gc this is because for each row it is creating new direct dictionary object for data type;
Solution: create one object and use the same.
Improvement tested with 2million rows:
~19 seconds to ~9 seconds

This closes #1381


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

Branch: refs/heads/streaming_ingest
Commit: 2cd22e1d682f29c54a2491deec18314e65a03f4e
Parents: 7f6b08a
Author: kumarvishal <ku...@gmail.com>
Authored: Mon Sep 25 17:06:21 2017 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Tue Sep 26 20:06:50 2017 +0530

----------------------------------------------------------------------
 .../executer/RowLevelFilterExecuterImpl.java    | 27 +++++++++++++++-----
 1 file changed, 21 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2cd22e1d/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 b79f18d..8f3eb93 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
@@ -100,6 +100,16 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    */
   protected boolean isNaturalSorted;
 
+  /**
+   * date direct dictionary generator
+   */
+  private DirectDictionaryGenerator dateDictionaryGenerator;
+
+  /**
+   * timestamp direct dictionary generator
+   */
+  private DirectDictionaryGenerator timestampDictionaryGenerator;
+
   public RowLevelFilterExecuterImpl(List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, SegmentProperties segmentProperties,
@@ -132,6 +142,10 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
     this.exp = exp;
     this.tableIdentifier = tableIdentifier;
     this.complexDimensionInfoMap = complexDimensionInfoMap;
+    this.dateDictionaryGenerator =
+        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataType.DATE);
+    this.timestampDictionaryGenerator =
+        DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(DataType.TIMESTAMP);
     initDimensionBlockIndexes();
     initMeasureBlockIndexes();
   }
@@ -408,13 +422,14 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    */
   private Object getFilterActualValueFromDirectDictionaryValue(
       DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int dictionaryValue) {
-    Object memberString = null;
-    DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-        .getDirectDictionaryGenerator(dimColumnEvaluatorInfo.getDimension().getDataType());
-    if (null != directDictionaryGenerator) {
-      memberString = directDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
+    switch (dimColumnEvaluatorInfo.getDimension().getDataType()) {
+      case DATE:
+        return dateDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
+      case TIMESTAMP:
+        return timestampDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
+      default:
+        throw new RuntimeException("Invalid data type for dierct dictionary");
     }
-    return memberString;
   }
 
   /**


[40/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
deleted file mode 100644
index 25c8071..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.partition.api.impl;
-
-import java.util.HashMap;
-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.spark.partition.api.DataPartitioner;
-import org.apache.carbondata.spark.partition.api.Partition;
-
-
-public final class QueryPartitionHelper {
-  private static QueryPartitionHelper instance = new QueryPartitionHelper();
-  private Map<String, DataPartitioner> partitionerMap =
-      new HashMap<String, DataPartitioner>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  private Map<String, DefaultLoadBalancer> loadBalancerMap =
-      new HashMap<String, DefaultLoadBalancer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  private QueryPartitionHelper() {
-
-  }
-
-  public static QueryPartitionHelper getInstance() {
-    return instance;
-  }
-
-  /**
-   * Get partitions applicable for query based on filters applied in query
-   */
-  public List<Partition> getPartitionsForQuery(CarbonQueryPlan queryPlan) {
-    String tableUniqueName = queryPlan.getDatabaseName() + '_' + queryPlan.getTableName();
-
-    DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
-
-    return dataPartitioner.getPartitions();
-  }
-
-  public List<Partition> getAllPartitions(String databaseName, String tableName) {
-    String tableUniqueName = databaseName + '_' + tableName;
-
-    DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
-
-    return dataPartitioner.getAllPartitions();
-  }
-
-  /**
-   * Get the node name where the partition is assigned to.
-   */
-  public String getLocation(Partition partition, String databaseName, String tableName) {
-    String tableUniqueName = databaseName + '_' + tableName;
-
-    DefaultLoadBalancer loadBalancer = loadBalancerMap.get(tableUniqueName);
-    return loadBalancer.getNodeForPartitions(partition);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
deleted file mode 100644
index 0762799..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.partition.api.impl;
-
-import java.util.List;
-
-import org.apache.carbondata.spark.partition.api.DataPartitioner;
-import org.apache.carbondata.spark.partition.api.Partition;
-
-/**
- * Sample partition.
- */
-public class SampleDataPartitionerImpl implements DataPartitioner {
-
-  @Override
-  public List<Partition> getAllPartitions() {
-    return null;
-  }
-
-  @Override
-  public List<Partition> getPartitions() {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/splits/TableSplit.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/splits/TableSplit.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/splits/TableSplit.java
deleted file mode 100644
index 733aae9..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/splits/TableSplit.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.splits;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-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.spark.partition.api.Partition;
-
-import org.apache.hadoop.io.Writable;
-
-
-/**
- * It represents one region server as one split.
- */
-public class TableSplit implements Serializable, Writable {
-  private static final long serialVersionUID = -8058151330863145575L;
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(TableSplit.class.getName());
-  private List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  private Partition partition;
-
-  /**
-   * @return the locations
-   */
-  public List<String> getLocations() {
-    return locations;
-  }
-
-  /**
-   * @param locations the locations to set
-   */
-  public void setLocations(List<String> locations) {
-    this.locations = locations;
-  }
-
-  /**
-   * @return Returns the partitions.
-   */
-  public Partition getPartition() {
-    return partition;
-  }
-
-  /**
-   * @param partition The partitions to set.
-   */
-  public void setPartition(Partition partition) {
-    this.partition = partition;
-  }
-
-  @Override public void readFields(DataInput in) throws IOException {
-
-    int sizeLoc = in.readInt();
-    for (int i = 0; i < sizeLoc; i++) {
-      byte[] b = new byte[in.readInt()];
-      in.readFully(b);
-      locations.add(new String(b, Charset.defaultCharset()));
-    }
-
-    byte[] buf = new byte[in.readInt()];
-    in.readFully(buf);
-    ByteArrayInputStream bis = new ByteArrayInputStream(buf);
-    ObjectInputStream ois = new ObjectInputStream(bis);
-    try {
-      partition = (Partition) ois.readObject();
-    } catch (ClassNotFoundException e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    ois.close();
-  }
-
-  @Override public void write(DataOutput out) throws IOException {
-
-    int sizeLoc = locations.size();
-    out.writeInt(sizeLoc);
-    for (int i = 0; i < sizeLoc; i++) {
-      byte[] bytes = locations.get(i).getBytes(Charset.defaultCharset());
-      out.writeInt(bytes.length);
-      out.write(bytes);
-    }
-
-    ByteArrayOutputStream bos = new ByteArrayOutputStream();
-
-    ObjectOutputStream obs = new ObjectOutputStream(bos);
-    obs.writeObject(partition);
-    obs.close();
-    byte[] byteArray = bos.toByteArray();
-    out.writeInt(byteArray.length);
-    out.write(byteArray);
-  }
-
-  public String toString() {
-    return partition.getUniqueID() + ' ' + locations;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/CarbonQueryUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/CarbonQueryUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/CarbonQueryUtil.java
deleted file mode 100644
index 1e39edf..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/CarbonQueryUtil.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-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.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.spark.partition.api.Partition;
-import org.apache.carbondata.spark.partition.api.impl.DefaultLoadBalancer;
-import org.apache.carbondata.spark.partition.api.impl.PartitionMultiFileImpl;
-import org.apache.carbondata.spark.partition.api.impl.QueryPartitionHelper;
-import org.apache.carbondata.spark.splits.TableSplit;
-
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * This utilty parses the Carbon query plan to actual query model object.
- */
-public class CarbonQueryUtil {
-
-  private CarbonQueryUtil() {
-
-  }
-
-  /**
-   * It creates the one split for each region server.
-   */
-  public static synchronized TableSplit[] getTableSplits(String databaseName, String tableName,
-      CarbonQueryPlan queryPlan) {
-
-    //Just create splits depends on locations of region servers
-    List<Partition> allPartitions = null;
-    if (queryPlan == null) {
-      allPartitions =
-          QueryPartitionHelper.getInstance().getAllPartitions(databaseName, tableName);
-    } else {
-      allPartitions =
-          QueryPartitionHelper.getInstance().getPartitionsForQuery(queryPlan);
-    }
-    TableSplit[] splits = new TableSplit[allPartitions.size()];
-    for (int i = 0; i < splits.length; i++) {
-      splits[i] = new TableSplit();
-      List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-      Partition partition = allPartitions.get(i);
-      String location = QueryPartitionHelper.getInstance()
-          .getLocation(partition, databaseName, tableName);
-      locations.add(location);
-      splits[i].setPartition(partition);
-      splits[i].setLocations(locations);
-    }
-
-    return splits;
-  }
-
-  /**
-   * It creates the one split for each region server.
-   */
-  public static TableSplit[] getTableSplitsForDirectLoad(String sourcePath) {
-
-    //Just create splits depends on locations of region servers
-    DefaultLoadBalancer loadBalancer = null;
-    List<Partition> allPartitions = getAllFilesForDataLoad(sourcePath);
-    loadBalancer = new DefaultLoadBalancer(new ArrayList<String>(), allPartitions);
-    TableSplit[] tblSplits = new TableSplit[allPartitions.size()];
-    for (int i = 0; i < tblSplits.length; i++) {
-      tblSplits[i] = new TableSplit();
-      List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-      Partition partition = allPartitions.get(i);
-      String location = loadBalancer.getNodeForPartitions(partition);
-      locations.add(location);
-      tblSplits[i].setPartition(partition);
-      tblSplits[i].setLocations(locations);
-    }
-    return tblSplits;
-  }
-
-  /**
-   * split sourcePath by comma
-   */
-  public static void splitFilePath(String sourcePath, List<String> partitionsFiles,
-      String separator) {
-    if (StringUtils.isNotEmpty(sourcePath)) {
-      String[] files = sourcePath.split(separator);
-      Collections.addAll(partitionsFiles, files);
-    }
-  }
-
-  private static List<Partition> getAllFilesForDataLoad(String sourcePath) {
-    List<String> files = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    splitFilePath(sourcePath, files, CarbonCommonConstants.COMMA);
-    List<Partition> partitionList =
-        new ArrayList<Partition>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    Map<Integer, List<String>> partitionFiles = new HashMap<Integer, List<String>>();
-
-    partitionFiles.put(0, new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN));
-    partitionList.add(new PartitionMultiFileImpl(0 + "", partitionFiles.get(0)));
-
-    for (int i = 0; i < files.size(); i++) {
-      partitionFiles.get(0).add(files.get(i));
-    }
-    return partitionList;
-  }
-
-  public static List<String> getListOfSlices(LoadMetadataDetails[] details) {
-    List<String> slices = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    if (null != details) {
-      for (LoadMetadataDetails oneLoad : details) {
-        if (!CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals(oneLoad.getLoadStatus())) {
-          String loadName = CarbonCommonConstants.LOAD_FOLDER + oneLoad.getLoadName();
-          slices.add(loadName);
-        }
-      }
-    }
-    return slices;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/LoadMetadataUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/LoadMetadataUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/LoadMetadataUtil.java
deleted file mode 100644
index 91a9556..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/LoadMetadataUtil.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.util;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-
-/**
- * Utility for load data
- */
-public final class LoadMetadataUtil {
-  private LoadMetadataUtil() {
-
-  }
-
-  public static boolean isLoadDeletionRequired(String metaDataLocation) {
-    LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metaDataLocation);
-    if (details != null && details.length != 0) {
-      for (LoadMetadataDetails oneRow : details) {
-        if ((CarbonCommonConstants.MARKED_FOR_DELETE.equalsIgnoreCase(oneRow.getLoadStatus())
-            || CarbonCommonConstants.COMPACTED.equalsIgnoreCase(oneRow.getLoadStatus()))
-            && oneRow.getVisibility().equalsIgnoreCase("true")) {
-          return true;
-        }
-      }
-    }
-
-    return false;
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/Util.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/Util.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/Util.java
new file mode 100644
index 0000000..8c14cd3
--- /dev/null
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/Util.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.spark.util;
+
+import org.apache.spark.SparkConf;
+import org.apache.spark.util.Utils;
+
+public class Util {
+  /**
+   * return the Array of available local-dirs
+   */
+  public static String[] getConfiguredLocalDirs(SparkConf conf) {
+    return Utils.getConfiguredLocalDirs(conf);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
index 6ad10f3..0bc9285 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/CarbonOption.scala
@@ -37,7 +37,7 @@ class CarbonOption(options: Map[String, String]) {
 
   def partitionClass: String = {
     options.getOrElse("partitionClass",
-      "org.apache.carbondata.spark.partition.api.impl.SampleDataPartitionerImpl")
+      "org.apache.carbondata.processing.partition.impl.SampleDataPartitionerImpl")
   }
 
   def tempCSV: Boolean = options.getOrElse("tempCSV", "true").toBoolean

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala
index f7758a6..8cdccdc 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/PartitionFactory.scala
@@ -22,7 +22,7 @@ import org.apache.spark.Partitioner
 import org.apache.carbondata.core.metadata.schema.PartitionInfo
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.scan.partition.{HashPartitioner => JavaHashPartitioner, ListPartitioner => JavaListPartitioner, RangePartitioner => JavaRangePartitioner}
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
 
 object PartitionFactory {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 fed8a96..bdf9a71 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
@@ -33,10 +33,10 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.row.CarbonRow
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.processing.csvload.{CSVInputFormat, StringArrayWritable}
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
-import org.apache.carbondata.processing.sortandgroupby.sortdata.{NewRowComparator, NewRowComparatorForNormalDims, SortParameters}
+import org.apache.carbondata.processing.loading.{DataLoadProcessBuilder, FailureCauses}
+import org.apache.carbondata.processing.loading.csvinput.{CSVInputFormat, StringArrayWritable}
+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.CommonUtil
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 6943dcb..574fb8a 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
@@ -27,17 +27,19 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException
 import org.apache.carbondata.core.datastore.row.CarbonRow
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.processing.csvload.StringArrayWritable
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
-import org.apache.carbondata.processing.newflow.converter.impl.RowConverterImpl
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
-import org.apache.carbondata.processing.newflow.parser.impl.RowParserImpl
-import org.apache.carbondata.processing.newflow.sort.SortStepRowUtil
-import org.apache.carbondata.processing.newflow.steps.{DataConverterProcessorStepImpl, DataWriterProcessorStepImpl}
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters
+import org.apache.carbondata.processing.loading.DataLoadProcessBuilder
+import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl
+import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable
+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.steps.{DataConverterProcessorStepImpl, DataWriterProcessorStepImpl}
+import org.apache.carbondata.processing.sort.sortdata.SortParameters
 import org.apache.carbondata.processing.store.{CarbonFactHandler, CarbonFactHandlerFactory}
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.rdd.{NewRddIterator, StringArrayRow}
+import org.apache.carbondata.spark.util.Util
 
 object DataLoadProcessorStepOnSpark {
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
@@ -215,7 +217,7 @@ object DataLoadProcessorStepOnSpark {
     val carbonUseLocalDir = CarbonProperties.getInstance()
       .getProperty("carbon.use.local.dir", "false")
     if (carbonUseLocalDir.equalsIgnoreCase("true")) {
-      val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
+      val storeLocations = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
       if (null != storeLocations && storeLocations.nonEmpty) {
         storeLocation = storeLocations(Random.nextInt(storeLocations.length))
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortHelper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortHelper.scala
index 7880fee..a42680e 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortHelper.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/load/GlobalSortHelper.scala
@@ -21,8 +21,8 @@ import org.apache.spark.Accumulator
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.BadRecordsLogger
 
 object GlobalSortHelper {
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 a73b0df..8eb5101 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
@@ -21,8 +21,8 @@ import scala.collection.JavaConverters._
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.sort.SortScopeOptions
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 object ValidateUtil {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 6cf8a7a..5c6760a 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
@@ -28,10 +28,10 @@ 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.spliter.RowResultProcessor
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+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.load.CarbonLoaderUtil
+import org.apache.carbondata.spark.util.Util
 
 class AlterTableLoadPartitionRDD[K, V](alterPartitionModel: AlterPartitionModel,
     result: AlterPartitionResult[K, V],
@@ -78,7 +78,7 @@ class AlterTableLoadPartitionRDD[K, V](alterPartitionModel: AlterPartitionModel,
 
             if (carbonUseLocalDir.equalsIgnoreCase("true")) {
 
-                val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
+                val storeLocations = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
                 if (null != storeLocations && storeLocations.nonEmpty) {
                     storeLocation = storeLocations(Random.nextInt(storeLocations.length))
                 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 b63fc48..32523d8 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
@@ -24,8 +24,8 @@ import org.apache.spark.{Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.execution.command.Partitioner
 
+import org.apache.carbondata.processing.util.CarbonQueryUtil
 import org.apache.carbondata.spark.Value
-import org.apache.carbondata.spark.util.CarbonQueryUtil
 
 class CarbonCleanFilesRDD[V: ClassTag](
     sc: SparkContext,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 da391cf..45271a7 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
@@ -24,8 +24,8 @@ import org.apache.spark.rdd.RDD
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
+import org.apache.carbondata.processing.util.CarbonQueryUtil
 import org.apache.carbondata.spark.DeletedLoadResult
-import org.apache.carbondata.spark.util.CarbonQueryUtil
 
 class CarbonDeleteLoadByDateRDD[K, V](
     sc: SparkContext,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 9e43d0e..9a1ef33 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
@@ -24,8 +24,8 @@ import org.apache.spark.{Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.execution.command.Partitioner
 
+import org.apache.carbondata.processing.util.CarbonQueryUtil
 import org.apache.carbondata.spark.Value
-import org.apache.carbondata.spark.util.CarbonQueryUtil
 
 class CarbonDeleteLoadRDD[V: ClassTag](
     sc: SparkContext,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 d1d49b9..652720c 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
@@ -22,8 +22,8 @@ import scala.reflect.ClassTag
 import org.apache.spark.{Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 
+import org.apache.carbondata.processing.util.CarbonQueryUtil
 import org.apache.carbondata.spark.Value
-import org.apache.carbondata.spark.util.CarbonQueryUtil
 
 class CarbonDropTableRDD[V: ClassTag](
     sc: SparkContext,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 ca607e1..b2e0c47 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
@@ -34,8 +34,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.Row
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.dictionary.Dictionary
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
+import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
 import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
@@ -44,9 +43,9 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.service.{CarbonCommonFactory, PathService}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, CarbonUtil}
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.exception.NoRetryException
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
+import org.apache.carbondata.processing.loading.exception.NoRetryException
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.tasks.{DictionaryWriterTask, SortIndexWriterTask}
 import org.apache.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil}
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
index da0d082..16d0b80 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonIUDMergerRDD.scala
@@ -32,8 +32,8 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTable
 import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonMultiBlockSplit}
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger.CarbonDataMergerUtil
-import org.apache.carbondata.processing.model.CarbonLoadModel
 import org.apache.carbondata.spark.MergeResult
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 c8a55af..1f88f25 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
@@ -45,13 +45,12 @@ import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUt
 import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonMultiBlockSplit}
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.hadoop.util.{CarbonInputFormatUtil, CarbonInputSplitTaskInfo}
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.processing.merger._
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+import org.apache.carbondata.processing.splits.TableSplit
+import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil}
 import org.apache.carbondata.spark.MergeResult
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
-import org.apache.carbondata.spark.splits.TableSplit
-import org.apache.carbondata.spark.util.SparkDataTypeConverterImpl
+import org.apache.carbondata.spark.util.{SparkDataTypeConverterImpl, Util}
 
 class CarbonMergerRDD[K, V](
     sc: SparkContext,
@@ -91,7 +90,7 @@ class CarbonMergerRDD[K, V](
 
       if (carbonUseLocalDir.equalsIgnoreCase("true")) {
 
-        val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
+        val storeLocations = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
         if (null != storeLocations && storeLocations.nonEmpty) {
           storeLocation = storeLocations(Random.nextInt(storeLocations.length))
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
index 86bc79f..d38be0a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
@@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapred.JobConf
 import org.apache.hadoop.mapreduce.Job
-import org.apache.spark.{Partition, SparkContext, TaskContext}
+import org.apache.spark.{Partition, TaskContext}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.execution.command.AlterPartitionModel
 import org.apache.spark.sql.hive.DistributionUtil
@@ -46,9 +46,8 @@ import org.apache.carbondata.core.util.{ByteUtil, DataTypeUtil}
 import org.apache.carbondata.hadoop.{CarbonInputSplit, CarbonMultiBlockSplit}
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
 import org.apache.carbondata.processing.merger.CarbonCompactionUtil
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.spliter.CarbonSplitExecutor
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
+import org.apache.carbondata.processing.partition.spliter.CarbonSplitExecutor
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 1c08307..fc34127 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
@@ -41,8 +41,8 @@ import org.apache.carbondata.core.stats.{QueryStatistic, QueryStatisticsConstant
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, TaskMetricsMap}
 import org.apache.carbondata.hadoop._
 import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.InitInputMetrics
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
 import org.apache.carbondata.spark.util.SparkDataTypeConverterImpl
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
index c2b7b74..c2029e5 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/DataManagementFunc.scala
@@ -34,13 +34,13 @@ import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTable
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CompactionType}
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.{CarbonLoaderUtil, DeleteLoadFolders, LoadMetadataUtil}
 import org.apache.carbondata.spark._
 import org.apache.carbondata.spark.compaction.CompactionCallable
-import org.apache.carbondata.spark.load._
 import org.apache.carbondata.spark.partition.{DropPartitionCallable, SplitPartitionCallable}
-import org.apache.carbondata.spark.util.{CommonUtil, LoadMetadataUtil}
+import org.apache.carbondata.spark.util.CommonUtil
 
 /**
  * Common functions for data life cycle management

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 67c71a0..1d1b47a 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,16 +43,14 @@ import org.apache.carbondata.common.logging.impl.StandardLogService
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, ThreadLocalTaskInfo}
-import org.apache.carbondata.processing.csvload.BlockDetails
-import org.apache.carbondata.processing.csvload.CSVInputFormat
-import org.apache.carbondata.processing.csvload.CSVRecordReaderIterator
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.DataLoadExecutor
-import org.apache.carbondata.processing.newflow.exception.NoRetryException
+import org.apache.carbondata.processing.loading.{DataLoadExecutor, FailureCauses}
+import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInputFormat, CSVRecordReaderIterator}
+import org.apache.carbondata.processing.loading.exception.NoRetryException
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.splits.TableSplit
+import org.apache.carbondata.processing.util.{CarbonLoaderUtil, CarbonQueryUtil}
 import org.apache.carbondata.spark.DataLoadResult
-import org.apache.carbondata.spark.load.{CarbonLoaderUtil, FailureCauses}
-import org.apache.carbondata.spark.splits.TableSplit
-import org.apache.carbondata.spark.util.{CarbonQueryUtil, CarbonScalaUtil, CommonUtil}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
 
 class SerializableConfiguration(@transient var value: Configuration) extends Serializable {
 
@@ -149,7 +147,7 @@ class SparkPartitionLoader(model: CarbonLoadModel,
     val isCarbonUseMultiDir = CarbonProperties.getInstance().isUseMultiTempDir
 
     if (isCarbonUseLocalDir) {
-      val yarnStoreLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
+      val yarnStoreLocations = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
 
       if (!isCarbonUseMultiDir && null != yarnStoreLocations && yarnStoreLocations.nonEmpty) {
         // use single dir

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala
index f45dc83..4d782c9 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/UpdateDataLoad.scala
@@ -26,9 +26,9 @@ import org.apache.carbondata.common.CarbonIterator
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.DataLoadExecutor
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.DataLoadExecutor
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 /**
  * Data load in case of update command .

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 5040e69..df25a37 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,12 +50,11 @@ import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentSta
 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.processing.csvload.CSVInputFormat
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+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, CarbonLoaderUtil}
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
 
 object CommonUtil {
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 601c0c7..c121960 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
@@ -51,14 +51,13 @@ import org.apache.carbondata.core.service.CarbonCommonFactory
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil}
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter
-import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
-import org.apache.carbondata.processing.csvload.CSVInputFormat
-import org.apache.carbondata.processing.csvload.StringArrayWritable
-import org.apache.carbondata.processing.etl.DataLoadingException
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.exception.NoRetryException
+import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter
+import org.apache.carbondata.processing.exception.DataLoadingException
+import org.apache.carbondata.processing.loading.csvinput.{CSVInputFormat, StringArrayWritable}
+import org.apache.carbondata.processing.loading.exception.NoRetryException
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 import org.apache.carbondata.spark.CarbonSparkFactory
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
 import org.apache.carbondata.spark.rdd._
 import org.apache.carbondata.spark.tasks.{DictionaryWriterTask, SortIndexWriterTask}
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 42070c4..7d4dd49 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
@@ -40,7 +40,7 @@ import org.apache.carbondata.core.metadata.schema.PartitionInfo
 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, DataTypeUtil}
-import org.apache.carbondata.processing.newflow.sort.SortScopeOptions
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 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/349c59c7/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 fc20108..ebd1c6c 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
@@ -39,12 +39,11 @@ import org.apache.carbondata.core.service.CarbonCommonFactory
 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
-import org.apache.carbondata.processing.model.CarbonLoadModel
 import org.apache.carbondata.spark.CarbonSparkFactory
-import org.apache.carbondata.spark.load.FailureCauses
-import org.apache.carbondata.spark.rdd.AlterTableAddColumnRDD
-import org.apache.carbondata.spark.util.{DataTypeConverterUtil, GlobalDictionaryUtil}
+import org.apache.carbondata.spark.util.DataTypeConverterUtil
 
 case class TableModel(
     ifNotExistsSet: Boolean,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 afcd970..37b722f 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
@@ -28,7 +28,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.block.Distributable
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 object DistributionUtil {
   @transient

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/spark/util/FileUtils.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/FileUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/FileUtils.scala
index 3060ff8..7345d54 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/FileUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/FileUtils.scala
@@ -21,7 +21,7 @@ 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.processing.etl.DataLoadingException
+import org.apache.carbondata.processing.exception.DataLoadingException
 
 object FileUtils {
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 002ed27..dcda72b 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
@@ -36,7 +36,7 @@ import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.core.util.path.CarbonTablePath
 import org.apache.carbondata.hadoop.CarbonInputSplit
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
-import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.spark.util.CommonUtil
 
 object PartitionUtils {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala
index 643002d..9c37640 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/SparkUtil.scala
@@ -23,7 +23,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
 import org.apache.spark.{SparkContext, TaskContext}
 import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD}
 
-import org.apache.carbondata.processing.csvload.BlockDetails
+import org.apache.carbondata.processing.loading.csvinput.BlockDetails
 
 /*
  * this object use to handle file splits

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 596cebf..cf14a79 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -36,7 +36,7 @@ import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionCoalescer, N
 import org.apache.spark.sql.{CarbonEnv, DataFrame, Row, SQLContext}
 import org.apache.spark.sql.execution.command.{AlterTableModel, CompactionModel, ExecutionErrors, UpdateTableModel}
 import org.apache.spark.sql.hive.DistributionUtil
-import org.apache.spark.util.{FileUtils, SparkUtil}
+import org.apache.spark.util.SparkUtil
 
 import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -53,18 +53,18 @@ import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
 import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties}
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.processing.csvload.{BlockDetails, CSVInputFormat, StringArrayWritable}
-import org.apache.carbondata.processing.etl.DataLoadingException
+import org.apache.carbondata.processing.exception.DataLoadingException
+import org.apache.carbondata.processing.loading.{DataLoadProcessBuilder, FailureCauses}
+import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInputFormat, StringArrayWritable}
+import org.apache.carbondata.processing.loading.exception.{CarbonDataLoadingException, NoRetryException}
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionType}
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.exception.{BadRecordFoundException, CarbonDataLoadingException, NoRetryException}
-import org.apache.carbondata.processing.newflow.DataLoadProcessBuilder
-import org.apache.carbondata.processing.newflow.sort.SortScopeOptions
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+import org.apache.carbondata.processing.splits.TableSplit
+import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil, CarbonQueryUtil}
 import org.apache.carbondata.spark._
-import org.apache.carbondata.spark.load.{FailureCauses, _}
-import org.apache.carbondata.spark.splits.TableSplit
-import org.apache.carbondata.spark.util.{CarbonQueryUtil, CarbonScalaUtil, CommonUtil}
+import org.apache.carbondata.spark.load.DataLoadProcessBuilderOnSpark
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
 
 /**
  * This is the factory class which can create different RDD depends on user needs.
@@ -353,7 +353,7 @@ object CarbonDataRDDFactory {
           isCompactionTriggerByDDl
         )
         var storeLocation = ""
-        val configuredStore = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
+        val configuredStore = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
         if (null != configuredStore && configuredStore.nonEmpty) {
           storeLocation = configuredStore(Random.nextInt(configuredStore.length))
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
index c38f0e1..cb35960 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
@@ -43,9 +43,9 @@ import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
 import org.apache.carbondata.processing.exception.MultipleMatchingException
+import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CarbonDataMergerUtilResult, CompactionType}
 import org.apache.carbondata.spark.DeleteDelataResultImpl
-import org.apache.carbondata.spark.load.FailureCauses
 import org.apache.carbondata.spark.util.QueryPlanUtil
 
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 01cd113..9814cc2 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -46,16 +46,16 @@ import org.apache.carbondata.core.exception.InvalidConfigurationException
 import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, ICarbonLock, LockUsage}
 import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
 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.TableInfo
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.etl.DataLoadingException
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.newflow.exception.NoRetryException
+import org.apache.carbondata.processing.exception.DataLoadingException
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
+import org.apache.carbondata.processing.loading.exception.NoRetryException
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.TableOptionConstant
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.load.ValidateUtil
 import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DataManagementFunc, DictionaryLoadModel}
@@ -894,7 +894,7 @@ private[sql] case class DescribeCommandFormatted(
       (field.name, field.dataType.simpleString, comment)
     }
     val colPropStr = if (colProps.toString().trim().length() > 0) {
-      // drops additional comma at end
+      // drops additional comma at endpom
       colProps.toString().dropRight(1)
     } else {
       colProps.toString()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
index 6ed826f..b5b8261 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/AllDictionaryTestCase.scala
@@ -21,8 +21,8 @@ import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
 import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.TableOptionConstant
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 
 /**
   * Test Case for org.apache.carbondata.integration.spark.util.GlobalDictionaryUtil

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
index e2b185e..fd2ab2a 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
@@ -24,7 +24,7 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 /**
  * Utility for global dictionary test cases

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index 63f8c1f..f0de47d 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -21,11 +21,12 @@ package org.apache.carbondata.spark.util
 import org.apache.spark.sql.test.util.QueryTest
 import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
 import org.scalatest.BeforeAndAfterAll
+
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.etl.DataLoadingException
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.exception.DataLoadingException
+import org.apache.carbondata.processing.util.TableOptionConstant
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
index a126686..d1d31c1 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
@@ -31,8 +31,8 @@ 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.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.TableOptionConstant
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 
 class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAfterAll {
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
index 405322b..8864d3e 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
@@ -24,8 +24,8 @@ import org.scalatest.BeforeAndAfterAll
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.TableOptionConstant
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 
 /**
   * Test Case for org.apache.carbondata.spark.util.GlobalDictionaryUtil

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 5335fe2..5b76c79 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
@@ -54,17 +54,21 @@ import org.apache.carbondata.core.scan.partition.PartitionUtil
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatusManager}
 import org.apache.carbondata.core.util.{ByteUtil, CarbonProperties}
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.processing.csvload.{BlockDetails, CSVInputFormat, StringArrayWritable}
-import org.apache.carbondata.processing.etl.DataLoadingException
+import org.apache.carbondata.processing.exception.DataLoadingException
+import org.apache.carbondata.processing.loading.FailureCauses
+import org.apache.carbondata.processing.loading.csvinput.BlockDetails
+import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat
+import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
+import org.apache.carbondata.processing.loading.exception.NoRetryException
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions
 import org.apache.carbondata.processing.merger.{CarbonCompactionUtil, CarbonDataMergerUtil, CompactionType}
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.processing.newflow.exception.{CarbonDataLoadingException, NoRetryException}
-import org.apache.carbondata.processing.newflow.sort.SortScopeOptions
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil
+import org.apache.carbondata.processing.splits.TableSplit
+import org.apache.carbondata.processing.util.{CarbonDataProcessorUtil, CarbonLoaderUtil, CarbonQueryUtil}
 import org.apache.carbondata.spark.{DataLoadResultImpl, PartitionFactory, _}
-import org.apache.carbondata.spark.load.{FailureCauses, _}
-import org.apache.carbondata.spark.splits.TableSplit
-import org.apache.carbondata.spark.util.{CarbonQueryUtil, CarbonScalaUtil, CommonUtil}
+import org.apache.carbondata.spark.load._
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, Util}
 
 /**
  * This is the factory class which can create different RDD depends on user needs.
@@ -528,7 +532,7 @@ object CarbonDataRDDFactory {
           isCompactionTriggerByDDl
         )
         var storeLocation = ""
-        val configuredStore = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
+        val configuredStore = Util.getConfiguredLocalDirs(SparkEnv.get.conf)
         if (null != configuredStore && configuredStore.nonEmpty) {
           storeLocation = configuredStore(Random.nextInt(configuredStore.length))
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/QueryPlanUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/QueryPlanUtil.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/QueryPlanUtil.scala
deleted file mode 100644
index e0a8b58..0000000
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/QueryPlanUtil.scala
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.util
-
-import scala.reflect.ClassTag
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
-import org.apache.hadoop.mapred.JobConf
-import org.apache.hadoop.mapreduce.Job
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
-
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
-import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
-
-/**
- * All the utility functions for carbon plan creation
- */
-object QueryPlanUtil {
-
-  /**
-   * createCarbonInputFormat from query model
-   */
-  def createCarbonInputFormat(absoluteTableIdentifier: AbsoluteTableIdentifier) :
-  (CarbonTableInputFormat[Array[Object]], Job) = {
-    val carbonInputFormat = new CarbonTableInputFormat[Array[Object]]()
-    val jobConf: JobConf = new JobConf(new Configuration)
-    val job: Job = new Job(jobConf)
-    FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getTablePath))
-    (carbonInputFormat, job)
-  }
-
-  def createCarbonInputFormat[V: ClassTag](absoluteTableIdentifier: AbsoluteTableIdentifier,
-      conf: Configuration) : CarbonTableInputFormat[V] = {
-    val carbonInputFormat = new CarbonTableInputFormat[V]()
-    val job: Job = new Job(conf)
-    FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getTablePath))
-    carbonInputFormat
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
index 28c53a1..52df2a4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
index 897895a..9018f7b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
@@ -38,11 +38,11 @@ import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.format
-import org.apache.carbondata.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.etl.DataLoadingException
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.newflow.exception.NoRetryException
+import org.apache.carbondata.processing.exception.DataLoadingException
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
+import org.apache.carbondata.processing.loading.exception.NoRetryException
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.TableOptionConstant
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.load.ValidateUtil
 import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DictionaryLoadModel}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 02c7023..23318f3 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
@@ -20,7 +20,13 @@ package org.apache.spark.sql.execution.command.mutation
 import java.util
 
 import scala.collection.JavaConverters._
+import scala.reflect.ClassTag
 
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.mapreduce.Job
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
@@ -37,10 +43,10 @@ import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
 import org.apache.carbondata.hadoop.CarbonInputFormat
+import org.apache.carbondata.hadoop.api.CarbonTableInputFormat
 import org.apache.carbondata.processing.exception.MultipleMatchingException
+import org.apache.carbondata.processing.loading.FailureCauses
 import org.apache.carbondata.spark.DeleteDelataResultImpl
-import org.apache.carbondata.spark.load.FailureCauses
-import org.apache.carbondata.spark.util.QueryPlanUtil
 
 object DeleteExecution {
   val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
@@ -93,8 +99,7 @@ object DeleteExecution {
       dataRdd
     }
 
-    val (carbonInputFormat, job) =
-      QueryPlanUtil.createCarbonInputFormat(absoluteTableIdentifier)
+    val (carbonInputFormat, job) = createCarbonInputFormat(absoluteTableIdentifier)
     CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
     val keyRdd = deleteRdd.map({ row =>
       val tupleId: String = row
@@ -319,4 +324,21 @@ object DeleteExecution {
 
     true
   }
+
+  private def createCarbonInputFormat(absoluteTableIdentifier: AbsoluteTableIdentifier) :
+  (CarbonTableInputFormat[Array[Object]], Job) = {
+    val carbonInputFormat = new CarbonTableInputFormat[Array[Object]]()
+    val jobConf: JobConf = new JobConf(new Configuration)
+    val job: Job = new Job(jobConf)
+    FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getTablePath))
+    (carbonInputFormat, job)
+  }
+
+  private def createCarbonInputFormat[V: ClassTag](absoluteTableIdentifier: AbsoluteTableIdentifier,
+      conf: Configuration) : CarbonTableInputFormat[V] = {
+    val carbonInputFormat = new CarbonTableInputFormat[V]()
+    val job: Job = new Job(conf)
+    FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getTablePath))
+    carbonInputFormat
+  }
 }


[30/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ThreadStatusObserver.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ThreadStatusObserver.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ThreadStatusObserver.java
deleted file mode 100644
index 56a32a3..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ThreadStatusObserver.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort.impl;
-
-import java.util.concurrent.ExecutorService;
-
-public class ThreadStatusObserver {
-
-  /**
-   * lock object
-   */
-  private Object lock = new Object();
-
-  private ExecutorService executorService;
-
-  private Throwable throwable;
-
-  public ThreadStatusObserver(ExecutorService executorService) {
-    this.executorService = executorService;
-  }
-
-  public void notifyFailed(Throwable throwable) {
-    // Only the first failing thread should call for shutting down the executor service and
-    // should assign the throwable object else the actual cause for failure can be overridden as
-    // all the running threads will throw interrupted exception on calling shutdownNow and
-    // will override the throwable object
-    if (null == this.throwable) {
-      synchronized (lock) {
-        if (null == this.throwable) {
-          executorService.shutdownNow();
-          this.throwable = throwable;
-        }
-      }
-    }
-  }
-
-  public Throwable getThrowable() {
-    return throwable;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
deleted file mode 100644
index 056c96b..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
+++ /dev/null
@@ -1,338 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.sort.impl;
-
-import java.io.File;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-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.memory.MemoryException;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.row.CarbonSortBatch;
-import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeSortDataRows;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeIntermediateMerger;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeSingleThreadFinalSortFilesMerger;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * It parallely reads data from array of iterates and do merge sort.
- * It sorts data in batches and send to the next step.
- */
-public class UnsafeBatchParallelReadMergeSorterImpl extends AbstractMergeSorter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeBatchParallelReadMergeSorterImpl.class.getName());
-
-  private SortParameters sortParameters;
-
-  private ExecutorService executorService;
-
-  private AtomicLong rowCounter;
-
-  public UnsafeBatchParallelReadMergeSorterImpl(AtomicLong rowCounter) {
-    this.rowCounter = rowCounter;
-  }
-
-  @Override public void initialize(SortParameters sortParameters) {
-    this.sortParameters = sortParameters;
-
-  }
-
-  @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
-      throws CarbonDataLoadingException {
-    this.executorService = Executors.newFixedThreadPool(iterators.length);
-    this.threadStatusObserver = new ThreadStatusObserver(this.executorService);
-    int batchSize = CarbonProperties.getInstance().getBatchSize();
-    final SortBatchHolder sortBatchHolder = new SortBatchHolder(sortParameters, iterators.length,
-        this.threadStatusObserver);
-
-    try {
-      for (int i = 0; i < iterators.length; i++) {
-        executorService.execute(
-            new SortIteratorThread(iterators[i], sortBatchHolder, batchSize, rowCounter,
-                this.threadStatusObserver));
-      }
-    } catch (Exception e) {
-      checkError();
-      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
-    }
-    checkError();
-    // Creates the iterator to read from merge sorter.
-    Iterator<CarbonSortBatch> batchIterator = new CarbonIterator<CarbonSortBatch>() {
-
-      @Override public boolean hasNext() {
-        return sortBatchHolder.hasNext();
-      }
-
-      @Override public CarbonSortBatch next() {
-        return new CarbonSortBatch(sortBatchHolder.next());
-      }
-    };
-    return new Iterator[] { batchIterator };
-  }
-
-  @Override public void close() {
-    executorService.shutdown();
-    try {
-      executorService.awaitTermination(2, TimeUnit.DAYS);
-    } catch (InterruptedException e) {
-      LOGGER.error(e);
-    }
-  }
-
-  /**
-   * This thread iterates the iterator and adds the rows
-   */
-  private static class SortIteratorThread implements Runnable {
-
-    private Iterator<CarbonRowBatch> iterator;
-
-    private SortBatchHolder sortDataRows;
-
-    private Object[][] buffer;
-
-    private AtomicLong rowCounter;
-
-    private ThreadStatusObserver threadStatusObserver;
-
-    public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortBatchHolder sortDataRows,
-        int batchSize, AtomicLong rowCounter, ThreadStatusObserver threadStatusObserver) {
-      this.iterator = iterator;
-      this.sortDataRows = sortDataRows;
-      this.buffer = new Object[batchSize][];
-      this.rowCounter = rowCounter;
-      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) {
-              buffer[i++] = row.getData();
-            }
-          }
-          if (i > 0) {
-            synchronized (sortDataRows) {
-              sortDataRows.getSortDataRow().addRowBatchWithOutSync(buffer, i);
-              rowCounter.getAndAdd(i);
-              if (!sortDataRows.getSortDataRow().canAdd()) {
-                sortDataRows.finish(false);
-                sortDataRows.createSortDataRows();
-              }
-            }
-          }
-        }
-      } catch (Exception e) {
-        LOGGER.error(e);
-        this.threadStatusObserver.notifyFailed(e);
-      } finally {
-        sortDataRows.finishThread();
-      }
-    }
-
-  }
-
-  private static class SortBatchHolder
-      extends CarbonIterator<UnsafeSingleThreadFinalSortFilesMerger> {
-
-    private SortParameters sortParameters;
-
-    private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
-
-    private UnsafeIntermediateMerger unsafeIntermediateFileMerger;
-
-    private UnsafeSortDataRows sortDataRow;
-
-    private final BlockingQueue<UnsafeSingleThreadFinalSortFilesMerger> mergerQueue;
-
-    private AtomicInteger iteratorCount;
-
-    private int batchCount;
-
-    private ThreadStatusObserver threadStatusObserver;
-
-    private final Object lock = new Object();
-
-    public SortBatchHolder(SortParameters sortParameters, int numberOfThreads,
-        ThreadStatusObserver threadStatusObserver) {
-      this.sortParameters = sortParameters.getCopy();
-      this.iteratorCount = new AtomicInteger(numberOfThreads);
-      this.mergerQueue = new LinkedBlockingQueue<>(1);
-      this.threadStatusObserver = threadStatusObserver;
-      createSortDataRows();
-    }
-
-    private void createSortDataRows() {
-      int inMemoryChunkSizeInMB = CarbonProperties.getInstance().getSortMemoryChunkSizeInMB();
-      setTempLocation(sortParameters);
-      this.finalMerger = new UnsafeSingleThreadFinalSortFilesMerger(sortParameters,
-          sortParameters.getTempFileLocation());
-      unsafeIntermediateFileMerger = new UnsafeIntermediateMerger(sortParameters);
-      sortDataRow = new UnsafeSortDataRows(sortParameters, unsafeIntermediateFileMerger,
-          inMemoryChunkSizeInMB);
-
-      try {
-        sortDataRow.initialize();
-      } catch (MemoryException e) {
-        throw new CarbonDataLoadingException(e);
-      }
-      batchCount++;
-    }
-
-    private void setTempLocation(SortParameters parameters) {
-      String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
-          .getLocalDataFolderLocation(parameters.getDatabaseName(),
-            parameters.getTableName(), parameters.getTaskNo(), batchCount + "",
-            parameters.getSegmentId(), false, false);
-      String[] tempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
-          File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
-      parameters.setTempFileLocation(tempDirs);
-    }
-
-    @Override public UnsafeSingleThreadFinalSortFilesMerger next() {
-      try {
-        UnsafeSingleThreadFinalSortFilesMerger unsafeSingleThreadFinalSortFilesMerger =
-            mergerQueue.take();
-        if (unsafeSingleThreadFinalSortFilesMerger.isStopProcess()) {
-          throw new RuntimeException(threadStatusObserver.getThrowable());
-        }
-        return unsafeSingleThreadFinalSortFilesMerger;
-      } catch (InterruptedException e) {
-        throw new RuntimeException(e);
-      }
-    }
-
-    public UnsafeSortDataRows getSortDataRow() {
-      return sortDataRow;
-    }
-
-    public void finish(boolean isFinalAttempt) {
-      try {
-        // if the mergerQue is empty and some CarbonDataLoadingException exception has occurred
-        // then set stop process to true in the finalmerger instance
-        if (mergerQueue.isEmpty() && threadStatusObserver != null
-            && threadStatusObserver.getThrowable() != null && threadStatusObserver
-            .getThrowable() instanceof CarbonDataLoadingException) {
-          finalMerger.setStopProcess(true);
-          if (isFinalAttempt) {
-            iteratorCount.decrementAndGet();
-          }
-          mergerQueue.put(finalMerger);
-          return;
-        }
-        processRowToNextStep(sortDataRow, sortParameters);
-        unsafeIntermediateFileMerger.finish();
-        List<UnsafeCarbonRowPage> rowPages = unsafeIntermediateFileMerger.getRowPages();
-        finalMerger.startFinalMerge(rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]),
-            unsafeIntermediateFileMerger.getMergedPages());
-        unsafeIntermediateFileMerger.close();
-        if (isFinalAttempt) {
-          iteratorCount.decrementAndGet();
-        }
-        mergerQueue.put(finalMerger);
-        sortDataRow = null;
-        unsafeIntermediateFileMerger = null;
-        finalMerger = null;
-      } catch (CarbonDataWriterException e) {
-        throw new CarbonDataLoadingException(e);
-      } catch (CarbonSortKeyAndGroupByException e) {
-        throw new CarbonDataLoadingException(e);
-      } catch (InterruptedException e) {
-        // if fails to put in queue because of interrupted exception, we can offer to free the main
-        // thread from waiting.
-        if (finalMerger != null) {
-          finalMerger.setStopProcess(true);
-          boolean offered = mergerQueue.offer(finalMerger);
-          if (!offered) {
-            throw new CarbonDataLoadingException(e);
-          }
-        }
-        throw new CarbonDataLoadingException(e);
-      }
-    }
-
-    public void finishThread() {
-      synchronized (lock) {
-        if (iteratorCount.get() <= 1) {
-          finish(true);
-        } else {
-          iteratorCount.decrementAndGet();
-        }
-      }
-    }
-
-    public boolean hasNext() {
-      return iteratorCount.get() > 0 || !mergerQueue.isEmpty();
-    }
-
-    /**
-     * Below method will be used to process data to next step
-     */
-    private boolean processRowToNextStep(UnsafeSortDataRows sortDataRows, SortParameters parameters)
-        throws CarbonDataLoadingException {
-      if (null == sortDataRows) {
-        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 {
-        // start sorting
-        sortDataRows.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 (InterruptedException e) {
-        throw new CarbonDataLoadingException(e);
-      }
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
deleted file mode 100644
index a0d43ba..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterImpl.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.sort.impl;
-
-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.datastore.exception.CarbonDataWriterException;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeSortDataRows;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeIntermediateMerger;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeSingleThreadFinalSortFilesMerger;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-
-/**
- * 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.
- */
-public class UnsafeParallelReadMergeSorterImpl extends AbstractMergeSorter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeParallelReadMergeSorterImpl.class.getName());
-
-  private SortParameters sortParameters;
-
-  private UnsafeIntermediateMerger unsafeIntermediateFileMerger;
-
-  private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
-
-  private AtomicLong rowCounter;
-
-  public UnsafeParallelReadMergeSorterImpl(AtomicLong rowCounter) {
-    this.rowCounter = rowCounter;
-  }
-
-  @Override public void initialize(SortParameters sortParameters) {
-    this.sortParameters = sortParameters;
-    unsafeIntermediateFileMerger = new UnsafeIntermediateMerger(sortParameters);
-
-    finalMerger = new UnsafeSingleThreadFinalSortFilesMerger(sortParameters,
-        sortParameters.getTempFileLocation());
-  }
-
-  @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
-      throws CarbonDataLoadingException {
-    int inMemoryChunkSizeInMB = CarbonProperties.getInstance().getSortMemoryChunkSizeInMB();
-    UnsafeSortDataRows sortDataRow =
-        new UnsafeSortDataRows(sortParameters, unsafeIntermediateFileMerger, inMemoryChunkSizeInMB);
-    final int batchSize = CarbonProperties.getInstance().getBatchSize();
-    try {
-      sortDataRow.initialize();
-    } catch (MemoryException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
-    this.threadStatusObserver = new ThreadStatusObserver(executorService);
-
-    try {
-      for (int i = 0; i < iterators.length; i++) {
-        executorService.execute(
-            new SortIteratorThread(iterators[i], sortDataRow, batchSize, rowCounter,
-                this.threadStatusObserver));
-      }
-      executorService.shutdown();
-      executorService.awaitTermination(2, TimeUnit.DAYS);
-      processRowToNextStep(sortDataRow, sortParameters);
-    } catch (Exception e) {
-      checkError();
-      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
-    }
-    checkError();
-    try {
-      unsafeIntermediateFileMerger.finish();
-      List<UnsafeCarbonRowPage> rowPages = unsafeIntermediateFileMerger.getRowPages();
-      finalMerger.startFinalMerge(rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]),
-          unsafeIntermediateFileMerger.getMergedPages());
-    } catch (CarbonDataWriterException e) {
-      throw new CarbonDataLoadingException(e);
-    } catch (CarbonSortKeyAndGroupByException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-
-    // Creates the iterator to read from merge sorter.
-    Iterator<CarbonRowBatch> batchIterator = new CarbonIterator<CarbonRowBatch>() {
-
-      @Override public boolean hasNext() {
-        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;
-      }
-    };
-    return new Iterator[] { batchIterator };
-  }
-
-  @Override public void close() {
-    unsafeIntermediateFileMerger.close();
-    finalMerger.clear();
-  }
-
-  /**
-   * Below method will be used to process data to next step
-   */
-  private boolean processRowToNextStep(UnsafeSortDataRows sortDataRows, SortParameters parameters)
-      throws CarbonDataLoadingException {
-    if (null == sortDataRows) {
-      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 {
-      // start sorting
-      sortDataRows.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 (InterruptedException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-  }
-
-  /**
-   * This thread iterates the iterator and adds the rows
-   */
-  private static class SortIteratorThread implements Runnable {
-
-    private Iterator<CarbonRowBatch> iterator;
-
-    private UnsafeSortDataRows sortDataRows;
-
-    private Object[][] buffer;
-
-    private AtomicLong rowCounter;
-
-    private ThreadStatusObserver threadStatusObserver;
-
-    public SortIteratorThread(Iterator<CarbonRowBatch> iterator,
-        UnsafeSortDataRows sortDataRows, int batchSize, AtomicLong rowCounter,
-        ThreadStatusObserver threadStatusObserver) {
-      this.iterator = iterator;
-      this.sortDataRows = sortDataRows;
-      this.buffer = new Object[batchSize][];
-      this.rowCounter = rowCounter;
-      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) {
-              buffer[i++] = row.getData();
-            }
-          }
-          if (i > 0) {
-            sortDataRows.addRowBatch(buffer, i);
-            rowCounter.getAndAdd(i);
-          }
-        }
-      } catch (Exception e) {
-        LOGGER.error(e);
-        this.threadStatusObserver.notifyFailed(e);
-      }
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
deleted file mode 100644
index 54e0180..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.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.newflow.DataField;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeSortDataRows;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeIntermediateMerger;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeSingleThreadFinalSortFilesMerger;
-import org.apache.carbondata.processing.sortandgroupby.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(String.valueOf(i), 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);
-    // 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.getPartitionID(), 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 String partitionId;
-
-    private int batchSize;
-
-    private boolean firstRow;
-
-    private UnsafeIntermediateMerger intermediateMerger;
-
-    public MergedDataIterator(String partitionId, int batchSize,
-        UnsafeIntermediateMerger intermediateMerger) {
-      this.partitionId = partitionId;
-      this.batchSize = batchSize;
-      this.intermediateMerger = intermediateMerger;
-      this.firstRow = true;
-    }
-
-    private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
-
-    @Override public boolean hasNext() {
-      if (firstRow) {
-        firstRow = false;
-        finalMerger = getFinalMerger(partitionId);
-        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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/UnsafeCarbonRowPage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/UnsafeCarbonRowPage.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/UnsafeCarbonRowPage.java
deleted file mode 100644
index 8b23437..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/UnsafeCarbonRowPage.java
+++ /dev/null
@@ -1,405 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.sort.unsafe;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-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.core.metadata.datatype.DataType;
-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;
-
-  private long sizeToBeUsed;
-
-  private MemoryBlock dataBlock;
-
-  private boolean saveToDisk;
-
-  private MemoryManagerType managerType;
-
-  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;
-    this.saveToDisk = saveToDisk;
-    this.nullSetWords = new long[((measureSize - 1) >> 6) + 1];
-    this.taskId = taskId;
-    buffer = new IntPointerBuffer(this.taskId);
-    this.dataBlock = memoryBlock;
-    // TODO Only using 98% of space for safe side.May be we can have different logic.
-    sizeToBeUsed = dataBlock.size() - (dataBlock.size() * 5) / 100;
-    this.managerType = MemoryManagerType.UNSAFE_MEMORY_MANAGER;
-  }
-
-  public int addRow(Object[] row) {
-    int size = addRow(row, dataBlock.getBaseOffset() + lastSize);
-    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) {
-        switch (measureDataType[mesCount]) {
-          case SHORT:
-            Short sval = (Short) value;
-            CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, sval);
-            size += 2;
-            break;
-          case INT:
-            Integer ival = (Integer) value;
-            CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, ival);
-            size += 4;
-            break;
-          case LONG:
-            Long val = (Long) value;
-            CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, val);
-            size += 8;
-            break;
-          case DOUBLE:
-            Double doubleVal = (Double) value;
-            CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, doubleVal);
-            size += 8;
-            break;
-          case DECIMAL:
-            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;
-            break;
-          default:
-            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;
-  }
-
-  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)) {
-        switch (measureDataType[mesCount]) {
-          case SHORT:
-            Short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-            size += 2;
-            rowToFill[dimensionSize + mesCount] = sval;
-            break;
-          case INT:
-            Integer ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-            size += 4;
-            rowToFill[dimensionSize + mesCount] = ival;
-            break;
-          case LONG:
-            Long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
-            size += 8;
-            rowToFill[dimensionSize + mesCount] = val;
-            break;
-          case DOUBLE:
-            Double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
-            size += 8;
-            rowToFill[dimensionSize + mesCount] = doubleVal;
-            break;
-          case DECIMAL:
-            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);
-            break;
-          default:
-            throw new IllegalArgumentException("unsupported data type:" +
-                measureDataType[mesCount]);
-        }
-      } else {
-        rowToFill[dimensionSize + mesCount] = null;
-      }
-    }
-    return rowToFill;
-  }
-
-  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)) {
-        switch (measureDataType[mesCount]) {
-          case SHORT:
-            short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
-            size += 2;
-            stream.writeShort(sval);
-            break;
-          case INT:
-            int ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
-            size += 4;
-            stream.writeInt(ival);
-            break;
-          case LONG:
-            long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
-            size += 8;
-            stream.writeLong(val);
-            break;
-          case DOUBLE:
-            double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
-            size += 8;
-            stream.writeDouble(doubleVal);
-            break;
-          case DECIMAL:
-            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);
-            break;
-          default:
-            throw new IllegalArgumentException("unsupported data type:" +
-                measureDataType[mesCount]);
-        }
-      }
-    }
-  }
-
-  public void freeMemory() {
-    switch (managerType) {
-      case UNSAFE_MEMORY_MANAGER:
-        UnsafeMemoryManager.INSTANCE.freeMemory(taskId, dataBlock);
-        break;
-      default:
-        UnsafeSortMemoryManager.INSTANCE.freeMemory(taskId, dataBlock);
-        buffer.freeMemory();
-    }
-  }
-
-  public boolean isSaveToDisk() {
-    return saveToDisk;
-  }
-
-  public IntPointerBuffer getBuffer() {
-    return buffer;
-  }
-
-  public int getUsedSize() {
-    return lastSize;
-  }
-
-  public boolean canAdd() {
-    return lastSize < sizeToBeUsed;
-  }
-
-  public MemoryBlock getDataBlock() {
-    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 void setNewDataBlock(MemoryBlock newMemoryBlock) {
-    this.dataBlock = newMemoryBlock;
-    this.managerType = MemoryManagerType.UNSAFE_SORT_MEMORY_MANAGER;
-  }
-
-  public enum MemoryManagerType {
-    UNSAFE_MEMORY_MANAGER, UNSAFE_SORT_MEMORY_MANAGER
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/UnsafeSortDataRows.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/UnsafeSortDataRows.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/UnsafeSortDataRows.java
deleted file mode 100644
index dda0d89..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/UnsafeSortDataRows.java
+++ /dev/null
@@ -1,413 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.sort.unsafe;
-
-import java.io.BufferedOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-
-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.memory.CarbonUnsafe;
-import org.apache.carbondata.core.memory.IntPointerBuffer;
-import org.apache.carbondata.core.memory.MemoryBlock;
-import org.apache.carbondata.core.memory.MemoryException;
-import org.apache.carbondata.core.memory.UnsafeMemoryManager;
-import org.apache.carbondata.core.memory.UnsafeSortMemoryManager;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.ThreadLocalTaskInfo;
-import org.apache.carbondata.processing.newflow.sort.unsafe.comparator.UnsafeRowComparator;
-import org.apache.carbondata.processing.newflow.sort.unsafe.comparator.UnsafeRowComparatorForNormalDIms;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeCarbonRow;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeIntermediateMerger;
-import org.apache.carbondata.processing.newflow.sort.unsafe.sort.TimSort;
-import org.apache.carbondata.processing.newflow.sort.unsafe.sort.UnsafeIntSortDataFormat;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class UnsafeSortDataRows {
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeSortDataRows.class.getName());
-  /**
-   * threadStatusObserver
-   */
-  private ThreadStatusObserver threadStatusObserver;
-  /**
-   * executor service for data sort holder
-   */
-  private ExecutorService dataSorterAndWriterExecutorService;
-  /**
-   * semaphore which will used for managing sorted data object arrays
-   */
-
-  private SortParameters parameters;
-
-  private UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger;
-
-  private UnsafeCarbonRowPage rowPage;
-
-  private final Object addRowsLock = new Object();
-
-  private long inMemoryChunkSize;
-
-  private boolean enableInMemoryIntermediateMerge;
-
-  private int bytesAdded;
-
-  private long maxSizeAllowed;
-
-  /**
-   * semaphore which will used for managing sorted data object arrays
-   */
-  private Semaphore semaphore;
-
-  private final long taskId;
-
-  public UnsafeSortDataRows(SortParameters parameters,
-      UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger, int inMemoryChunkSize) {
-    this.parameters = parameters;
-
-    this.unsafeInMemoryIntermediateFileMerger = unsafeInMemoryIntermediateFileMerger;
-
-    // observer of writing file in thread
-    this.threadStatusObserver = new ThreadStatusObserver();
-    this.taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
-    this.inMemoryChunkSize = inMemoryChunkSize;
-    this.inMemoryChunkSize = inMemoryChunkSize * 1024L * 1024L;
-    enableInMemoryIntermediateMerge = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.ENABLE_INMEMORY_MERGE_SORT,
-            CarbonCommonConstants.ENABLE_INMEMORY_MERGE_SORT_DEFAULT));
-
-    this.maxSizeAllowed = parameters.getBatchSortSizeinMb();
-    if (maxSizeAllowed <= 0) {
-      // If user does not input any memory size, then take half the size of usable memory configured
-      // in sort memory size.
-      this.maxSizeAllowed = UnsafeMemoryManager.INSTANCE.getUsableMemory() / 2;
-    } else {
-      this.maxSizeAllowed = this.maxSizeAllowed * 1024 * 1024;
-    }
-  }
-
-  /**
-   * This method will be used to initialize
-   */
-  public void initialize() throws MemoryException {
-    MemoryBlock baseBlock =
-        UnsafeMemoryManager.allocateMemoryWithRetry(this.taskId, inMemoryChunkSize);
-    boolean isMemoryAvailable =
-        UnsafeSortMemoryManager.INSTANCE.isMemoryAvailable(baseBlock.size());
-    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);
-    // Delete if any older file exists in sort temp folder
-    deleteSortLocationIfExists();
-
-    // create new sort temp directory
-    CarbonDataProcessorUtil.createLocations(parameters.getTempFileLocation());
-    this.dataSorterAndWriterExecutorService =
-        Executors.newFixedThreadPool(parameters.getNumberOfCores());
-    semaphore = new Semaphore(parameters.getNumberOfCores());
-  }
-
-  public boolean canAdd() {
-    return bytesAdded < maxSizeAllowed;
-  }
-
-  /**
-   * This method will be used to add new row
-   *
-   * @param rowBatch new rowBatch
-   * @throws CarbonSortKeyAndGroupByException problem while writing
-   */
-  public void addRowBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
-    // 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
-    synchronized (addRowsLock) {
-      addBatch(rowBatch, size);
-    }
-  }
-
-  /**
-   * This method will be used to add new row
-   *
-   * @param rowBatch new rowBatch
-   * @param size
-   * @throws CarbonSortKeyAndGroupByException problem while writing
-   */
-  public void addRowBatchWithOutSync(Object[][] rowBatch, int size)
-      throws CarbonSortKeyAndGroupByException {
-    // 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
-    addBatch(rowBatch, size);
-  }
-
-  private void addBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
-    for (int i = 0; i < size; i++) {
-      if (rowPage.canAdd()) {
-        bytesAdded += rowPage.addRow(rowBatch[i]);
-      } else {
-        try {
-          if (enableInMemoryIntermediateMerge) {
-            unsafeInMemoryIntermediateFileMerger.startInmemoryMergingIfPossible();
-          }
-          unsafeInMemoryIntermediateFileMerger.startFileMergingIfPossible();
-          semaphore.acquire();
-          dataSorterAndWriterExecutorService.execute(new DataSorterAndWriter(rowPage));
-          MemoryBlock memoryBlock =
-              UnsafeMemoryManager.allocateMemoryWithRetry(this.taskId, inMemoryChunkSize);
-          boolean saveToDisk =
-              UnsafeSortMemoryManager.INSTANCE.isMemoryAvailable(memoryBlock.size());
-          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]);
-        } catch (Exception e) {
-          LOGGER.error(
-                  "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
-          throw new CarbonSortKeyAndGroupByException(e);
-        }
-
-      }
-    }
-  }
-
-  /**
-   * This method will be used to add new row
-   */
-  public void addRow(Object[] row) throws CarbonSortKeyAndGroupByException {
-    // 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);
-    } else {
-      try {
-        if (enableInMemoryIntermediateMerge) {
-          unsafeInMemoryIntermediateFileMerger.startInmemoryMergingIfPossible();
-        }
-        unsafeInMemoryIntermediateFileMerger.startFileMergingIfPossible();
-        semaphore.acquire();
-        dataSorterAndWriterExecutorService.submit(new DataSorterAndWriter(rowPage));
-        MemoryBlock memoryBlock =
-            UnsafeMemoryManager.allocateMemoryWithRetry(this.taskId, inMemoryChunkSize);
-        boolean saveToDisk = UnsafeSortMemoryManager.INSTANCE.isMemoryAvailable(memoryBlock.size());
-        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);
-      } catch (Exception e) {
-        LOGGER.error(
-            "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
-        throw new CarbonSortKeyAndGroupByException(e);
-      }
-
-    }
-  }
-
-  /**
-   * Below method will be used to start storing process This method will get
-   * all the temp files present in sort temp folder then it will create the
-   * record holder heap and then it will read first record from each file and
-   * initialize the heap
-   *
-   * @throws InterruptedException
-   */
-  public void startSorting() throws InterruptedException {
-    LOGGER.info("Unsafe based sorting will be used");
-    if (this.rowPage.getUsedSize() > 0) {
-      TimSort<UnsafeCarbonRow, IntPointerBuffer> timSort = new TimSort<>(
-          new UnsafeIntSortDataFormat(rowPage));
-      if (parameters.getNumberOfNoDictSortColumns() > 0) {
-        timSort.sort(rowPage.getBuffer(), 0, rowPage.getBuffer().getActualSize(),
-            new UnsafeRowComparator(rowPage));
-      } else {
-        timSort.sort(rowPage.getBuffer(), 0, rowPage.getBuffer().getActualSize(),
-            new UnsafeRowComparatorForNormalDIms(rowPage));
-      }
-      unsafeInMemoryIntermediateFileMerger.addDataChunkToMerge(rowPage);
-    } else {
-      rowPage.freeMemory();
-    }
-    startFileBasedMerge();
-  }
-
-  private void writeData(UnsafeCarbonRowPage rowPage, File file)
-      throws CarbonSortKeyAndGroupByException {
-    DataOutputStream stream = null;
-    try {
-      // open stream
-      stream = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(file),
-          parameters.getFileWriteBufferSize()));
-      int actualSize = rowPage.getBuffer().getActualSize();
-      // 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);
-      }
-
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
-    } finally {
-      // close streams
-      CarbonUtil.closeStreams(stream);
-    }
-  }
-
-  /**
-   * This method will be used to delete sort temp location is it is exites
-   */
-  public void deleteSortLocationIfExists() {
-    CarbonDataProcessorUtil.deleteSortLocationIfExists(parameters.getTempFileLocation());
-  }
-
-  /**
-   * Below method will be used to start file based merge
-   *
-   * @throws InterruptedException
-   */
-  private void startFileBasedMerge() throws InterruptedException {
-    dataSorterAndWriterExecutorService.shutdown();
-    dataSorterAndWriterExecutorService.awaitTermination(2, TimeUnit.DAYS);
-  }
-
-  /**
-   * Observer class for thread execution
-   * In case of any failure we need stop all the running thread
-   */
-  private class ThreadStatusObserver {
-    /**
-     * Below method will be called if any thread fails during execution
-     *
-     * @param exception
-     * @throws CarbonSortKeyAndGroupByException
-     */
-    public void notifyFailed(Throwable exception) throws CarbonSortKeyAndGroupByException {
-      dataSorterAndWriterExecutorService.shutdownNow();
-      unsafeInMemoryIntermediateFileMerger.close();
-      parameters.getObserver().setFailed(true);
-      LOGGER.error(exception);
-      throw new CarbonSortKeyAndGroupByException(exception);
-    }
-  }
-
-  /**
-   * This class is responsible for sorting and writing the object
-   * array which holds the records equal to given array size
-   */
-  private class DataSorterAndWriter implements Runnable {
-    private UnsafeCarbonRowPage page;
-
-    public DataSorterAndWriter(UnsafeCarbonRowPage rowPage) {
-      this.page = rowPage;
-    }
-
-    @Override
-    public void run() {
-      try {
-        long startTime = System.currentTimeMillis();
-        TimSort<UnsafeCarbonRow, IntPointerBuffer> timSort = new TimSort<>(
-            new UnsafeIntSortDataFormat(page));
-        // if sort_columns is not none, sort by sort_columns
-        if (parameters.getNumberOfNoDictSortColumns() > 0) {
-          timSort.sort(page.getBuffer(), 0, page.getBuffer().getActualSize(),
-              new UnsafeRowComparator(page));
-        } else {
-          timSort.sort(page.getBuffer(), 0, page.getBuffer().getActualSize(),
-              new UnsafeRowComparatorForNormalDIms(page));
-        }
-        if (page.isSaveToDisk()) {
-          // create a new file every time
-          // 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);
-          writeData(page, sortTempFile);
-          LOGGER.info("Time taken to sort row page with size" + page.getBuffer().getActualSize()
-              + " and write is: " + (System.currentTimeMillis() - startTime));
-          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
-          unsafeInMemoryIntermediateFileMerger.addFileToMerge(sortTempFile);
-        } else {
-          // creating a new memory block as size is already allocated
-          // so calling lazy memory allocator
-          MemoryBlock newMemoryBlock = UnsafeSortMemoryManager.INSTANCE
-              .allocateMemoryLazy(taskId, page.getDataBlock().size());
-          // copying data from working memory manager to sortmemory manager
-          CarbonUnsafe.getUnsafe()
-              .copyMemory(page.getDataBlock().getBaseObject(), page.getDataBlock().getBaseOffset(),
-                  newMemoryBlock.getBaseObject(), newMemoryBlock.getBaseOffset(),
-                  page.getDataBlock().size());
-          // free unsafememory manager
-          page.freeMemory();
-          page.setNewDataBlock(newMemoryBlock);
-          // add sort temp filename to and arrayList. When the list size reaches 20 then
-          // intermediate merging of sort temp files will be triggered
-          page.getBuffer().loadToUnsafe();
-          unsafeInMemoryIntermediateFileMerger.addDataChunkToMerge(page);
-          LOGGER.info(
-              "Time taken to sort row page with size" + page.getBuffer().getActualSize() + "is: "
-                  + (System.currentTimeMillis() - startTime));
-        }
-      } catch (Throwable e) {
-        try {
-          threadStatusObserver.notifyFailed(e);
-        } catch (CarbonSortKeyAndGroupByException ex) {
-          LOGGER.error(e);
-        }
-      } finally {
-        semaphore.release();
-      }
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/comparator/UnsafeRowComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/comparator/UnsafeRowComparator.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/comparator/UnsafeRowComparator.java
deleted file mode 100644
index c54dcd6..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/comparator/UnsafeRowComparator.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort.unsafe.comparator;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.memory.CarbonUnsafe;
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeCarbonRow;
-
-public class UnsafeRowComparator implements Comparator<UnsafeCarbonRow> {
-
-  /**
-   * mapping of dictionary and no dictionary of sort_columns.
-   */
-  private boolean[] noDictionarySortColumnMaping;
-
-  private Object baseObject;
-
-  public UnsafeRowComparator(UnsafeCarbonRowPage rowPage) {
-    this.noDictionarySortColumnMaping = rowPage.getNoDictionarySortColumnMapping();
-    this.baseObject = rowPage.getDataBlock().getBaseObject();
-  }
-
-  /**
-   * 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;
-  }
-
-  /**
-   * Below method will be used to compare two mdkey
-   */
-  public int compare(UnsafeCarbonRow rowL, Object baseObjectL, UnsafeCarbonRow rowR,
-      Object baseObjectR) {
-    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(baseObjectL, rowA + sizeA);
-        byte[] byteArr1 = new byte[aShort1];
-        sizeA += 2;
-        CarbonUnsafe.getUnsafe()
-            .copyMemory(baseObjectL, rowA + sizeA, byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET,
-                aShort1);
-        sizeA += aShort1;
-
-        short aShort2 = CarbonUnsafe.getUnsafe().getShort(baseObjectR, rowB + sizeB);
-        byte[] byteArr2 = new byte[aShort2];
-        sizeB += 2;
-        CarbonUnsafe.getUnsafe()
-            .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 + sizeA);
-        sizeA += 4;
-        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObjectR, rowB + sizeB);
-        sizeB += 4;
-        diff = dimFieldA - dimFieldB;
-        if (diff != 0) {
-          return diff;
-        }
-      }
-    }
-
-    return diff;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/comparator/UnsafeRowComparatorForNormalDIms.java
deleted file mode 100644
index 53f976f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/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.newflow.sort.unsafe.comparator;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.core.memory.CarbonUnsafe;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/SortTempChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/SortTempChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/SortTempChunkHolder.java
deleted file mode 100644
index 9eab940..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/SortTempChunkHolder.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort.unsafe.holder;
-
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-/**
- * Interface for merging temporary sort files/ inmemory data
- */
-public interface SortTempChunkHolder extends Comparable<SortTempChunkHolder> {
-
-  boolean hasNext();
-
-  void readRow()  throws CarbonSortKeyAndGroupByException;
-
-  Object[] getRow();
-
-  int numberOfRows();
-
-  void close();
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeCarbonRow.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeCarbonRow.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeCarbonRow.java
deleted file mode 100644
index aff60f6..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeCarbonRow.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.processing.newflow.sort.unsafe.holder;
-
-public class UnsafeCarbonRow {
-
-  public long address;
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeCarbonRowForMerge.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeCarbonRowForMerge.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeCarbonRowForMerge.java
deleted file mode 100644
index 0ec4553..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeCarbonRowForMerge.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.processing.newflow.sort.unsafe.holder;
-
-public class UnsafeCarbonRowForMerge extends UnsafeCarbonRow {
-
-  public byte index;
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeFinalMergePageHolder.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
deleted file mode 100644
index f00dd45..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort.unsafe.holder;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeInMemoryIntermediateDataMerger;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.NewRowComparator;
-
-public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(UnsafeFinalMergePageHolder.class.getName());
-
-  private int counter;
-
-  private int actualSize;
-
-  private long[] mergedAddresses;
-
-  private byte[] rowPageIndexes;
-
-  private UnsafeCarbonRowPage[] rowPages;
-
-  private NewRowComparator comparator;
-
-  private Object[] currentRow;
-
-  private int columnSize;
-
-  public UnsafeFinalMergePageHolder(UnsafeInMemoryIntermediateDataMerger merger,
-      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 NewRowComparator(noDictSortColumnMapping);
-    this.columnSize = columnSize;
-  }
-
-  public boolean hasNext() {
-    if (counter < actualSize) {
-      return true;
-    }
-    return false;
-  }
-
-  public void readRow() {
-    currentRow = new Object[columnSize];
-    rowPages[rowPageIndexes[counter]].getRow(mergedAddresses[counter], currentRow);
-    counter++;
-  }
-
-  public Object[] getRow() {
-    return currentRow;
-  }
-
-  @Override public int compareTo(SortTempChunkHolder o) {
-    return comparator.compare(currentRow, o.getRow());
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-
-    if (!(obj instanceof UnsafeFinalMergePageHolder)) {
-      return false;
-    }
-
-    UnsafeFinalMergePageHolder o = (UnsafeFinalMergePageHolder) obj;
-    return this == o;
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode();
-  }
-
-  public int numberOfRows() {
-    return actualSize;
-  }
-
-  public void close() {
-    for (int i = 0; i < rowPages.length; i++) {
-      rowPages[i].freeMemory();
-    }
-  }
-}


[11/50] [abbrv] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
deleted file mode 100644
index 5820b9d..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/IUDCommands.scala
+++ /dev/null
@@ -1,857 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command
-
-import java.util
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hive.CarbonRelation
-import org.apache.spark.storage.StorageLevel
-
-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.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.mutate.{CarbonUpdateUtil, DeleteDeltaBlockDetails, SegmentUpdateDetails, TupleIdEnum}
-import org.apache.carbondata.core.mutate.data.RowCountDetailsVO
-import org.apache.carbondata.core.statusmanager.{SegmentStatusManager, SegmentUpdateStatusManager}
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
-import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
-import org.apache.carbondata.hadoop.CarbonInputFormat
-import org.apache.carbondata.processing.exception.MultipleMatchingException
-import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CarbonDataMergerUtilResult, CompactionType}
-import org.apache.carbondata.spark.DeleteDelataResultImpl
-import org.apache.carbondata.spark.load.FailureCauses
-import org.apache.carbondata.spark.util.QueryPlanUtil
-
-
-/**
- * IUD update delete and compaction framework.
- *
- */
-
-private[sql] case class ProjectForDeleteCommand(
-     plan: LogicalPlan,
-     identifier: Seq[String],
-     timestamp: String) extends RunnableCommand with DataProcessCommand {
-
-  var horizontalCompactionFailed = false
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val dataFrame = Dataset.ofRows(sparkSession, plan)
-    //    dataFrame.show(truncate = false)
-    //    dataFrame.collect().foreach(println)
-    val dataRdd = dataFrame.rdd
-
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(deleteExecution.getTableIdentifier(identifier))(sparkSession).
-        asInstanceOf[CarbonRelation]
-    val carbonTable = relation.tableMeta.carbonTable
-    val metadataLock = CarbonLockFactory
-        .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-          LockUsage.METADATA_LOCK)
-    var lockStatus = false
-    try {
-      lockStatus = metadataLock.lockWithRetries()
-      LOGGER.audit(s" Delete data request has been received " +
-          s"for ${ relation.databaseName }.${ relation.tableName }.")
-      if (lockStatus) {
-        LOGGER.info("Successfully able to get the table metadata file lock")
-      }
-      else {
-        throw new Exception("Table is locked for deletion. Please try after some time")
-      }
-      val tablePath = CarbonStorePath.getCarbonTablePath(
-        carbonTable.getStorePath,
-        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier)
-      var executorErrors = new ExecutionErrors(FailureCauses.NONE, "")
-
-      // handle the clean up of IUD.
-      CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, false)
-
-      if (deleteExecution
-          .deleteDeltaExecution(identifier, sparkSession, dataRdd, timestamp, relation,
-            false, executorErrors)) {
-        // call IUD Compaction.
-        IUDCommon.tryHorizontalCompaction(sparkSession, relation, isUpdateOperation = false)
-      }
-    } catch {
-      case e: HorizontalCompactionException =>
-        LOGGER.error("Delete operation passed. Exception in Horizontal Compaction." +
-            " Please check logs. " + e.getMessage)
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, e.compactionTimeStamp.toString)
-
-      case e: Exception =>
-        LOGGER.error(e, "Exception in Delete data operation " + e.getMessage)
-        // ****** start clean up.
-        // In case of failure , clean all related delete delta files
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
-
-        // clean up. Null check is required as for executor error some times message is null
-        if (null != e.getMessage) {
-          sys.error("Delete data operation is failed. " + e.getMessage)
-        }
-        else {
-          sys.error("Delete data operation is failed. Please check logs.")
-        }
-    } finally {
-      if (lockStatus) {
-        CarbonLockUtil.fileUnlock(metadataLock, LockUsage.METADATA_LOCK)
-      }
-    }
-    Seq.empty
-  }
-}
-
-private[sql] case class ProjectForUpdateCommand(
-    plan: LogicalPlan, tableIdentifier: Seq[String]) extends RunnableCommand
-    with DataProcessCommand {
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(ProjectForUpdateCommand.getClass.getName)
-
-    //  sqlContext.sparkContext.setLocalProperty(org.apache.spark.sql.execution.SQLExecution
-    //  .EXECUTION_ID_KEY, null)
-    // DataFrame(sqlContext, plan).show(truncate = false)
-    // return Seq.empty
-
-
-    val res = plan find {
-      case relation: LogicalRelation if relation.relation
-          .isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        true
-      case _ => false
-    }
-
-    if (res.isEmpty) {
-      return Seq.empty
-    }
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(deleteExecution.getTableIdentifier(tableIdentifier))(sparkSession).
-        asInstanceOf[CarbonRelation]
-    //    val relation = CarbonEnv.get.carbonMetastore
-    //      .lookupRelation1(deleteExecution.getTableIdentifier(tableIdentifier))(sqlContext).
-    //      asInstanceOf[CarbonRelation]
-    val carbonTable = relation.tableMeta.carbonTable
-    val metadataLock = CarbonLockFactory
-        .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
-          LockUsage.METADATA_LOCK)
-    var lockStatus = false
-    // get the current time stamp which should be same for delete and update.
-    val currentTime = CarbonUpdateUtil.readCurrentTime
-    //    var dataFrame: DataFrame = null
-    var dataSet: DataFrame = null
-    var isPersistEnabled = CarbonProperties.getInstance.isPersistUpdateDataset()
-    try {
-      lockStatus = metadataLock.lockWithRetries()
-      if (lockStatus) {
-        logInfo("Successfully able to get the table metadata file lock")
-      }
-      else {
-        throw new Exception("Table is locked for updation. Please try after some time")
-      }
-      val tablePath = CarbonStorePath.getCarbonTablePath(
-        carbonTable.getStorePath,
-        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier)
-      // Get RDD.
-
-      dataSet = if (isPersistEnabled) {
-        Dataset.ofRows(sparkSession, plan).persist(StorageLevel.fromString(
-          CarbonProperties.getInstance.getUpdateDatasetStorageLevel()))
-      }
-      else {
-        Dataset.ofRows(sparkSession, plan)
-      }
-      var executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
-
-
-      // handle the clean up of IUD.
-      CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, false)
-
-      // do delete operation.
-      deleteExecution.deleteDeltaExecution(tableIdentifier, sparkSession, dataSet.rdd,
-        currentTime + "",
-        relation, isUpdateOperation = true, executionErrors)
-
-      if(executionErrors.failureCauses != FailureCauses.NONE) {
-        throw new Exception(executionErrors.errorMsg)
-      }
-
-      // do update operation.
-      UpdateExecution.performUpdate(dataSet, tableIdentifier, plan,
-        sparkSession, currentTime, executionErrors)
-
-      if(executionErrors.failureCauses != FailureCauses.NONE) {
-        throw new Exception(executionErrors.errorMsg)
-      }
-
-      // Do IUD Compaction.
-      IUDCommon.tryHorizontalCompaction(sparkSession, relation, isUpdateOperation = true)
-    } catch {
-      case e: HorizontalCompactionException =>
-        LOGGER.error(
-          "Update operation passed. Exception in Horizontal Compaction. Please check logs." + e)
-        // In case of failure , clean all related delta files
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, e.compactionTimeStamp.toString)
-
-      case e: Exception =>
-        LOGGER.error("Exception in update operation" + e)
-        // ****** start clean up.
-        // In case of failure , clean all related delete delta files
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, currentTime + "")
-
-        // *****end clean up.
-        if (null != e.getMessage) {
-          sys.error("Update operation failed. " + e.getMessage)
-        }
-        if (null != e.getCause && null != e.getCause.getMessage) {
-          sys.error("Update operation failed. " + e.getCause.getMessage)
-        }
-        sys.error("Update operation failed. please check logs.")
-    }
-    finally {
-      if (null != dataSet && isPersistEnabled) {
-        dataSet.unpersist()
-      }
-      if (lockStatus) {
-        CarbonLockUtil.fileUnlock(metadataLock, LockUsage.METADATA_LOCK)
-      }
-    }
-    Seq.empty
-  }
-}
-
-object IUDCommon {
-
-  val LOG = LogServiceFactory.getLogService(this.getClass.getName)
-
-  /**
-   * The method does horizontal compaction. After Update and Delete completion
-   * tryHorizontal compaction will be called. In case this method is called after
-   * Update statement then Update Compaction followed by Delete Compaction will be
-   * processed whereas for tryHorizontalCompaction called after Delete statement
-   * then only Delete Compaction will be processed.
-    *
-    * @param sparkSession
-   * @param carbonRelation
-   * @param isUpdateOperation
-   */
-  def tryHorizontalCompaction(sparkSession: SparkSession,
-      carbonRelation: CarbonRelation,
-      isUpdateOperation: Boolean): Unit = {
-
-    var ishorizontalCompaction = CarbonDataMergerUtil.isHorizontalCompactionEnabled()
-
-    if (ishorizontalCompaction == false) {
-      return
-    }
-
-    var compactionTypeIUD = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
-    val carbonTable = carbonRelation.tableMeta.carbonTable
-    val (db, table) = (carbonTable.getDatabaseName, carbonTable.getFactTableName)
-    val absTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    val updateTimeStamp = System.currentTimeMillis()
-    // To make sure that update and delete timestamps are not same,
-    // required to commit to status metadata and cleanup
-    val deleteTimeStamp = updateTimeStamp + 1
-
-    // get the valid segments
-    var segLists = CarbonDataMergerUtil.getValidSegmentList(absTableIdentifier)
-
-    if (segLists == null || segLists.size() == 0) {
-      return
-    }
-
-    // Should avoid reading Table Status file from Disk every time. Better to load it
-    // in-memory at the starting and pass it along the routines. The constructor of
-    // SegmentUpdateStatusManager reads the Table Status File and Table Update Status
-    // file and save the content in segmentDetails and updateDetails respectively.
-    val segmentUpdateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
-      absTableIdentifier)
-
-    if (isUpdateOperation == true) {
-
-      // This is only update operation, perform only update compaction.
-      compactionTypeIUD = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
-      performUpdateDeltaCompaction(sparkSession,
-        compactionTypeIUD,
-        carbonTable,
-        absTableIdentifier,
-        segmentUpdateStatusManager,
-        updateTimeStamp,
-        segLists)
-    }
-
-    // After Update Compaction perform delete compaction
-    compactionTypeIUD = CompactionType.IUD_DELETE_DELTA_COMPACTION
-    segLists = CarbonDataMergerUtil.getValidSegmentList(absTableIdentifier)
-    if (segLists == null || segLists.size() == 0) {
-      return
-    }
-
-    // Delete Compaction
-    performDeleteDeltaCompaction(sparkSession,
-      compactionTypeIUD,
-      carbonTable,
-      absTableIdentifier,
-      segmentUpdateStatusManager,
-      deleteTimeStamp,
-      segLists)
-  }
-
-  /**
-   * Update Delta Horizontal Compaction.
-    *
-    * @param sparkSession
-   * @param compactionTypeIUD
-   * @param carbonTable
-   * @param absTableIdentifier
-   * @param segLists
-   */
-  private def performUpdateDeltaCompaction(sparkSession: SparkSession,
-      compactionTypeIUD: CompactionType,
-      carbonTable: CarbonTable,
-      absTableIdentifier: AbsoluteTableIdentifier,
-      segmentUpdateStatusManager: SegmentUpdateStatusManager,
-      factTimeStamp: Long,
-      segLists: util.List[String]): Unit = {
-    val db = carbonTable.getDatabaseName
-    val table = carbonTable.getFactTableName
-    // get the valid segments qualified for update compaction.
-    val validSegList = CarbonDataMergerUtil.getSegListIUDCompactionQualified(segLists,
-      absTableIdentifier,
-      segmentUpdateStatusManager,
-      compactionTypeIUD)
-
-    if (validSegList.size() == 0) {
-      return
-    }
-
-    LOG.info(s"Horizontal Update Compaction operation started for [${db}.${table}].")
-    LOG.audit(s"Horizontal Update Compaction operation started for [${db}.${table}].")
-
-    try {
-      // Update Compaction.
-      val altertablemodel = AlterTableModel(Option(carbonTable.getDatabaseName),
-        carbonTable.getFactTableName,
-        Some(segmentUpdateStatusManager),
-        CompactionType.IUD_UPDDEL_DELTA_COMPACTION.toString,
-        Some(factTimeStamp),
-        "")
-
-      AlterTableCompaction(altertablemodel).run(sparkSession)
-    }
-    catch {
-      case e: Exception =>
-        val msg = if (null != e.getMessage) {
-          e.getMessage
-        } else {
-          "Please check logs for more info"
-        }
-        throw new HorizontalCompactionException(
-          s"Horizontal Update Compaction Failed for [${ db }.${ table }]. " + msg, factTimeStamp)
-    }
-    LOG.info(s"Horizontal Update Compaction operation completed for [${ db }.${ table }].")
-    LOG.audit(s"Horizontal Update Compaction operation completed for [${ db }.${ table }].")
-  }
-
-  /**
-   * Delete Delta Horizontal Compaction.
-    *
-    * @param sparkSession
-   * @param compactionTypeIUD
-   * @param carbonTable
-   * @param absTableIdentifier
-   * @param segLists
-   */
-  private def performDeleteDeltaCompaction(sparkSession: SparkSession,
-      compactionTypeIUD: CompactionType,
-      carbonTable: CarbonTable,
-      absTableIdentifier: AbsoluteTableIdentifier,
-      segmentUpdateStatusManager: SegmentUpdateStatusManager,
-      factTimeStamp: Long,
-      segLists: util.List[String]): Unit = {
-
-    val db = carbonTable.getDatabaseName
-    val table = carbonTable.getFactTableName
-    val deletedBlocksList = CarbonDataMergerUtil.getSegListIUDCompactionQualified(segLists,
-      absTableIdentifier,
-      segmentUpdateStatusManager,
-      compactionTypeIUD)
-
-    if (deletedBlocksList.size() == 0) {
-      return
-    }
-
-    LOG.info(s"Horizontal Delete Compaction operation started for [${db}.${table}].")
-    LOG.audit(s"Horizontal Delete Compaction operation started for [${db}.${table}].")
-
-    try {
-
-      // Delete Compaction RDD
-      val rdd1 = sparkSession.sparkContext
-        .parallelize(deletedBlocksList.asScala.toSeq, deletedBlocksList.size())
-
-      val timestamp = factTimeStamp
-      val updateStatusDetails = segmentUpdateStatusManager.getUpdateStatusDetails
-      val result = rdd1.mapPartitions(iter =>
-        new Iterator[Seq[CarbonDataMergerUtilResult]] {
-          override def hasNext: Boolean = iter.hasNext
-
-          override def next(): Seq[CarbonDataMergerUtilResult] = {
-            val segmentAndBlocks = iter.next
-            val segment = segmentAndBlocks.substring(0, segmentAndBlocks.lastIndexOf("/"))
-            val blockName = segmentAndBlocks
-              .substring(segmentAndBlocks.lastIndexOf("/") + 1, segmentAndBlocks.length)
-
-            val result = CarbonDataMergerUtil.compactBlockDeleteDeltaFiles(segment, blockName,
-              absTableIdentifier,
-              updateStatusDetails,
-              timestamp)
-
-            result.asScala.toList
-
-          }
-        }).collect
-
-      val resultList = ListBuffer[CarbonDataMergerUtilResult]()
-      result.foreach(x => {
-        x.foreach(y => {
-          resultList += y
-        })
-      })
-
-      val updateStatus = CarbonDataMergerUtil.updateStatusFile(resultList.toList.asJava,
-        carbonTable,
-        timestamp.toString,
-        segmentUpdateStatusManager)
-      if (updateStatus == false) {
-        LOG.audit(s"Delete Compaction data operation is failed for [${db}.${table}].")
-        LOG.error("Delete Compaction data operation is failed.")
-        throw new HorizontalCompactionException(
-          s"Horizontal Delete Compaction Failed for [${db}.${table}] ." +
-          s" Please check logs for more info.", factTimeStamp)
-      }
-      else {
-        LOG.info(s"Horizontal Delete Compaction operation completed for [${db}.${table}].")
-        LOG.audit(s"Horizontal Delete Compaction operation completed for [${db}.${table}].")
-      }
-    }
-    catch {
-      case e: Exception =>
-        val msg = if (null != e.getMessage) {
-          e.getMessage
-        } else {
-          "Please check logs for more info"
-        }
-        throw new HorizontalCompactionException(
-          s"Horizontal Delete Compaction Failed for [${ db }.${ table }]. " + msg, factTimeStamp)
-    }
-  }
-}
-
-class HorizontalCompactionException(
-    message: String,
-    // required for cleanup
-    val compactionTimeStamp: Long) extends RuntimeException(message) {
-}
-
-object deleteExecution {
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
-  def getTableIdentifier(tableIdentifier: Seq[String]): TableIdentifier = {
-    if (tableIdentifier.size > 1) {
-      TableIdentifier(tableIdentifier(1), Some(tableIdentifier(0)))
-    } else {
-      TableIdentifier(tableIdentifier(0), None)
-    }
-  }
-
-  def deleteDeltaExecution(identifier: Seq[String],
-                           sparkSession: SparkSession,
-                           dataRdd: RDD[Row],
-                           timestamp: String, relation: CarbonRelation, isUpdateOperation: Boolean,
-                           executorErrors: ExecutionErrors): Boolean = {
-
-    var res: Array[List[(String, (SegmentUpdateDetails, ExecutionErrors))]] = null
-    val tableName = getTableIdentifier(identifier).table
-    val database = getDB.getDatabaseName(getTableIdentifier(identifier).database, sparkSession)
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      .lookupRelation(deleteExecution.getTableIdentifier(identifier))(sparkSession).
-      asInstanceOf[CarbonRelation]
-
-    val storeLocation = relation.tableMeta.storePath
-    val absoluteTableIdentifier: AbsoluteTableIdentifier = new
-        AbsoluteTableIdentifier(storeLocation,
-          relation.tableMeta.carbonTableIdentifier)
-    var tablePath = CarbonStorePath
-      .getCarbonTablePath(storeLocation,
-        absoluteTableIdentifier.getCarbonTableIdentifier())
-    var tableUpdateStatusPath = tablePath.getTableUpdateStatusFilePath
-    val totalSegments =
-      SegmentStatusManager.readLoadMetadata(tablePath.getMetadataDirectoryPath).length
-    var factPath = tablePath.getFactDir
-
-    var carbonTable = relation.tableMeta.carbonTable
-    var deleteStatus = true
-    val deleteRdd = if (isUpdateOperation) {
-      val schema =
-        org.apache.spark.sql.types.StructType(Seq(org.apache.spark.sql.types.StructField(
-          CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID,
-          org.apache.spark.sql.types.StringType)))
-      val rdd = dataRdd
-        .map(row => Row(row.get(row.fieldIndex(
-          CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))))
-      sparkSession.createDataFrame(rdd, schema).rdd
-      // sqlContext.createDataFrame(rdd, schema).rdd
-    } else {
-      dataRdd
-    }
-
-    val (carbonInputFormat, job) =
-      QueryPlanUtil.createCarbonInputFormat(absoluteTableIdentifier)
-    CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
-    val keyRdd = deleteRdd.map({ row =>
-      val tupleId: String = row
-        .getString(row.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
-      val key = CarbonUpdateUtil.getSegmentWithBlockFromTID(tupleId)
-      (key, row)
-    }).groupByKey()
-
-    // if no loads are present then no need to do anything.
-    if (keyRdd.partitions.size == 0) {
-      return true
-    }
-
-    var blockMappingVO = carbonInputFormat.getBlockRowCount(job, absoluteTableIdentifier)
-    val segmentUpdateStatusMngr = new SegmentUpdateStatusManager(absoluteTableIdentifier)
-    CarbonUpdateUtil
-      .createBlockDetailsMap(blockMappingVO, segmentUpdateStatusMngr)
-
-    val rowContRdd =
-      sparkSession.sparkContext.parallelize(
-        blockMappingVO.getCompleteBlockRowDetailVO.asScala.toSeq,
-          keyRdd.partitions.length)
-
-//    val rowContRdd = sqlContext.sparkContext
-//      .parallelize(blockMappingVO.getCompleteBlockRowDetailVO.asScala.toSeq,
-//        keyRdd.partitions.size)
-
-    val rdd = rowContRdd.join(keyRdd)
-
-    // rdd.collect().foreach(println)
-
-    res = rdd.mapPartitionsWithIndex(
-      (index: Int, records: Iterator[((String), (RowCountDetailsVO, Iterable[Row]))]) =>
-        Iterator[List[(String, (SegmentUpdateDetails, ExecutionErrors))]] {
-
-          var result = List[(String, (SegmentUpdateDetails, ExecutionErrors))]()
-          while (records.hasNext) {
-            val ((key), (rowCountDetailsVO, groupedRows)) = records.next
-            result = result ++
-              deleteDeltaFunc(index,
-                key,
-                groupedRows.toIterator,
-                timestamp,
-                rowCountDetailsVO)
-
-          }
-          result
-        }
-    ).collect()
-
-    // if no loads are present then no need to do anything.
-    if (res.isEmpty) {
-      return true
-    }
-
-    // update new status file
-    checkAndUpdateStatusFiles
-
-    // all or none : update status file, only if complete delete opeartion is successfull.
-    def checkAndUpdateStatusFiles: Unit = {
-      val blockUpdateDetailsList = new util.ArrayList[SegmentUpdateDetails]()
-      val segmentDetails = new util.HashSet[String]()
-      res.foreach(resultOfSeg => resultOfSeg.foreach(
-        resultOfBlock => {
-          if (resultOfBlock._1.equalsIgnoreCase(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)) {
-            blockUpdateDetailsList.add(resultOfBlock._2._1)
-            segmentDetails.add(resultOfBlock._2._1.getSegmentName)
-            // if this block is invalid then decrement block count in map.
-            if (CarbonUpdateUtil.isBlockInvalid(resultOfBlock._2._1.getStatus)) {
-              CarbonUpdateUtil.decrementDeletedBlockCount(resultOfBlock._2._1,
-                blockMappingVO.getSegmentNumberOfBlockMapping)
-            }
-          }
-          else {
-            deleteStatus = false
-            // In case of failure , clean all related delete delta files
-            CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
-            LOGGER.audit(s"Delete data operation is failed for ${ database }.${ tableName }")
-            val errorMsg =
-              "Delete data operation is failed due to failure in creating delete delta file for " +
-                "segment : " + resultOfBlock._2._1.getSegmentName + " block : " +
-                resultOfBlock._2._1.getBlockName
-            executorErrors.failureCauses = resultOfBlock._2._2.failureCauses
-            executorErrors.errorMsg = resultOfBlock._2._2.errorMsg
-
-            if (executorErrors.failureCauses == FailureCauses.NONE) {
-              executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
-              executorErrors.errorMsg = errorMsg
-            }
-            LOGGER.error(errorMsg)
-            return
-          }
-        }
-      )
-      )
-
-      val listOfSegmentToBeMarkedDeleted = CarbonUpdateUtil
-        .getListOfSegmentsToMarkDeleted(blockMappingVO.getSegmentNumberOfBlockMapping)
-
-
-
-      // this is delete flow so no need of putting timestamp in the status file.
-      if (CarbonUpdateUtil
-        .updateSegmentStatus(blockUpdateDetailsList, carbonTable, timestamp, false) &&
-        CarbonUpdateUtil
-          .updateTableMetadataStatus(segmentDetails,
-            carbonTable,
-            timestamp,
-            !isUpdateOperation,
-            listOfSegmentToBeMarkedDeleted)
-      ) {
-        LOGGER.info(s"Delete data operation is successful for ${ database }.${ tableName }")
-        LOGGER.audit(s"Delete data operation is successful for ${ database }.${ tableName }")
-      }
-      else {
-        // In case of failure , clean all related delete delta files
-        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
-
-        val errorMessage = "Delete data operation is failed due to failure " +
-          "in table status updation."
-        LOGGER.audit(s"Delete data operation is failed for ${ database }.${ tableName }")
-        LOGGER.error("Delete data operation is failed due to failure in table status updation.")
-        executorErrors.failureCauses = FailureCauses.STATUS_FILE_UPDATION_FAILURE
-        executorErrors.errorMsg = errorMessage
-        // throw new Exception(errorMessage)
-      }
-    }
-
-    def deleteDeltaFunc(index: Int,
-                        key: String,
-                        iter: Iterator[Row],
-                        timestamp: String,
-                        rowCountDetailsVO: RowCountDetailsVO):
-    Iterator[(String, (SegmentUpdateDetails, ExecutionErrors))] = {
-
-      val result = new DeleteDelataResultImpl()
-      var deleteStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
-      val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-      // here key = segment/blockName
-      val blockName = CarbonUpdateUtil
-        .getBlockName(
-          CarbonTablePath.addDataPartPrefix(key.split(CarbonCommonConstants.FILE_SEPARATOR)(1)))
-      val segmentId = key.split(CarbonCommonConstants.FILE_SEPARATOR)(0)
-      var deleteDeltaBlockDetails: DeleteDeltaBlockDetails = new DeleteDeltaBlockDetails(blockName)
-      val resultIter = new Iterator[(String, (SegmentUpdateDetails, ExecutionErrors))] {
-        val segmentUpdateDetails = new SegmentUpdateDetails()
-        var TID = ""
-        var countOfRows = 0
-        try {
-          while (iter.hasNext) {
-            val oneRow = iter.next
-            TID = oneRow
-              .get(oneRow.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID)).toString
-            val offset = CarbonUpdateUtil.getRequiredFieldFromTID(TID, TupleIdEnum.OFFSET)
-            val blockletId = CarbonUpdateUtil
-              .getRequiredFieldFromTID(TID, TupleIdEnum.BLOCKLET_ID)
-            val pageId = Integer.parseInt(CarbonUpdateUtil
-              .getRequiredFieldFromTID(TID, TupleIdEnum.PAGE_ID))
-            val IsValidOffset = deleteDeltaBlockDetails.addBlocklet(blockletId, offset, pageId)
-            // stop delete operation
-            if(!IsValidOffset) {
-              executorErrors.failureCauses = FailureCauses.MULTIPLE_INPUT_ROWS_MATCHING
-              executorErrors.errorMsg = "Multiple input rows matched for same row."
-              throw new MultipleMatchingException("Multiple input rows matched for same row.")
-            }
-            countOfRows = countOfRows + 1
-          }
-
-          val blockPath = CarbonUpdateUtil.getTableBlockPath(TID, factPath)
-          val completeBlockName = CarbonTablePath
-            .addDataPartPrefix(CarbonUpdateUtil.getRequiredFieldFromTID(TID, TupleIdEnum.BLOCK_ID) +
-              CarbonCommonConstants.FACT_FILE_EXT)
-          val deleteDeletaPath = CarbonUpdateUtil
-            .getDeleteDeltaFilePath(blockPath, blockName, timestamp)
-          val carbonDeleteWriter = new CarbonDeleteDeltaWriterImpl(deleteDeletaPath,
-            FileFactory.getFileType(deleteDeletaPath))
-
-
-
-          segmentUpdateDetails.setBlockName(blockName)
-          segmentUpdateDetails.setActualBlockName(completeBlockName)
-          segmentUpdateDetails.setSegmentName(segmentId)
-          segmentUpdateDetails.setDeleteDeltaEndTimestamp(timestamp)
-          segmentUpdateDetails.setDeleteDeltaStartTimestamp(timestamp)
-
-          val alreadyDeletedRows: Long = rowCountDetailsVO.getDeletedRowsInBlock
-          val totalDeletedRows: Long = alreadyDeletedRows + countOfRows
-          segmentUpdateDetails.setDeletedRowsInBlock(totalDeletedRows.toString)
-          if (totalDeletedRows == rowCountDetailsVO.getTotalNumberOfRows) {
-            segmentUpdateDetails.setStatus(CarbonCommonConstants.MARKED_FOR_DELETE)
-          }
-          else {
-            // write the delta file
-            carbonDeleteWriter.write(deleteDeltaBlockDetails)
-          }
-
-          deleteStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
-        } catch {
-          case e : MultipleMatchingException =>
-            LOGGER.audit(e.getMessage)
-            LOGGER.error(e.getMessage)
-          // dont throw exception here.
-          case e: Exception =>
-            val errorMsg = s"Delete data operation is failed for ${ database }.${ tableName }."
-            LOGGER.audit(errorMsg)
-            LOGGER.error(errorMsg + e.getMessage)
-            throw e
-        }
-
-
-        var finished = false
-
-        override def hasNext: Boolean = {
-          if (!finished) {
-            finished = true
-            finished
-          }
-          else {
-            !finished
-          }
-        }
-
-        override def next(): (String, (SegmentUpdateDetails, ExecutionErrors)) = {
-          finished = true
-          result.getKey(deleteStatus, (segmentUpdateDetails, executorErrors))
-        }
-      }
-      resultIter
-    }
-    true
-  }
-}
-
-
-
-object UpdateExecution {
-
-  def performUpdate(
-         dataFrame: Dataset[Row],
-         tableIdentifier: Seq[String],
-         plan: LogicalPlan,
-         sparkSession: SparkSession,
-         currentTime: Long,
-         executorErrors: ExecutionErrors): Unit = {
-
-    def isDestinationRelation(relation: CarbonDatasourceHadoopRelation): Boolean = {
-
-      val tableName = relation.identifier.getCarbonTableIdentifier.getTableName
-      val dbName = relation.identifier.getCarbonTableIdentifier.getDatabaseName
-      (tableIdentifier.size > 1 &&
-        tableIdentifier(0) == dbName &&
-        tableIdentifier(1) == tableName) ||
-        (tableIdentifier(0) == tableName)
-    }
-    def getHeader(relation: CarbonDatasourceHadoopRelation, plan: LogicalPlan): String = {
-      var header = ""
-      var found = false
-
-      plan match {
-        case Project(pList, _) if (!found) =>
-          found = true
-          header = pList
-            .filter(field => !field.name
-              .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
-            .map(col => if (col.name.endsWith(CarbonCommonConstants.UPDATED_COL_EXTENSION)) {
-              col.name
-                .substring(0, col.name.lastIndexOf(CarbonCommonConstants.UPDATED_COL_EXTENSION))
-            }
-            else {
-              col.name
-            }).mkString(",")
-      }
-      header
-    }
-    val ex = dataFrame.queryExecution.analyzed
-    val res = ex find {
-      case relation: LogicalRelation
-        if relation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
-        isDestinationRelation(relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]) =>
-        true
-      case _ => false
-    }
-    val carbonRelation: CarbonDatasourceHadoopRelation = res match {
-      case Some(relation: LogicalRelation) =>
-        relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-      case _ => sys.error("")
-    }
-
-    val updateTableModel = UpdateTableModel(true, currentTime, executorErrors)
-
-    val header = getHeader(carbonRelation, plan)
-
-    LoadTable(
-      Some(carbonRelation.identifier.getCarbonTableIdentifier.getDatabaseName),
-      carbonRelation.identifier.getCarbonTableIdentifier.getTableName,
-      null,
-      Seq(),
-      Map(("fileheader" -> header)),
-      false,
-      null,
-      Some(dataFrame),
-      Some(updateTableModel)).run(sparkSession)
-
-    executorErrors.errorMsg = updateTableModel.executorErrors.errorMsg
-    executorErrors.failureCauses = updateTableModel.executorErrors.failureCauses
-
-    Seq.empty
-
-  }
-
-}


[39/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
index 9cf8a91..10e6785 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.hive.CarbonRelation
 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.spark.load.FailureCauses
+import org.apache.carbondata.processing.loading.FailureCauses
 
 /**
  * IUD update delete and compaction framework.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
index 036ca49..5e9d31f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
@@ -31,7 +31,7 @@ import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, Lock
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.spark.load.FailureCauses
+import org.apache.carbondata.processing.loading.FailureCauses
 
 private[sql] case class ProjectForUpdateCommand(
     plan: LogicalPlan, tableIdentifier: Seq[String])

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
index 1f06aed..e0b891a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
@@ -37,7 +37,7 @@ 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.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
 
 case class AlterTableDropCarbonPartitionCommand(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
index 21b974a..e16dfc9 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
@@ -39,7 +39,7 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
 import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 61589de..5c7d451 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,8 +25,8 @@ 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.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.TableOptionConstant
 
 /**
   * Test Case for org.apache.carbondata.integration.spark.util.GlobalDictionaryUtil

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 e7eb422..fd3b2cd 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
@@ -24,7 +24,7 @@ import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentif
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 /**
  * Utility for global dictionary test cases

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 4746ecf..399665f 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
@@ -27,9 +27,9 @@ 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.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.etl.DataLoadingException
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.exception.DataLoadingException
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.util.TableOptionConstant
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java
deleted file mode 100644
index 890534e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.api.dataloader;
-
-public class DataLoadModel {
-  /**
-   * Schema Info
-   */
-  private SchemaInfo schemaInfo;
-
-  /**
-   * table table
-   */
-  private String tableName;
-
-  /**
-   * is CSV load
-   */
-  private boolean isCsvLoad;
-
-  private String blocksID;
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-  /**
-   * new load start time
-   */
-  private String factTimeStamp;
-
-  private String escapeCharacter;
-
-  private String quoteCharacter;
-
-  private String commentCharacter;
-
-  private String rddIteratorKey;
-
-  private String dateFormat;
-
-  private String maxColumns;
-  /**
-   * @return Returns the schemaInfo.
-   */
-  public SchemaInfo getSchemaInfo() {
-    return schemaInfo;
-  }
-
-  /**
-   * @param schemaInfo The schemaInfo to set.
-   */
-  public void setSchemaInfo(SchemaInfo schemaInfo) {
-    this.schemaInfo = schemaInfo;
-  }
-
-  /**
-   * @return Returns the tableName.
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * @param tableName The tableName to set.
-   */
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  /**
-   * @return Returns the isCsvLoad.
-   */
-  public boolean isCsvLoad() {
-    return isCsvLoad;
-  }
-
-  /**
-   * @param isCsvLoad The isCsvLoad to set.
-   */
-  public void setCsvLoad(boolean isCsvLoad) {
-    this.isCsvLoad = isCsvLoad;
-  }
-
-  /**
-   * get block id
-   *
-   * @return
-   */
-  public String getBlocksID() {
-    return blocksID;
-  }
-
-  /**
-   * set block id to data load model
-   *
-   * @param blocksID
-   */
-  public void setBlocksID(String blocksID) {
-    this.blocksID = blocksID;
-  }
-
-  /**
-   * @return
-   */
-  public String getTaskNo() {
-    return taskNo;
-  }
-
-  /**
-   * @param taskNo
-   */
-  public void setTaskNo(String taskNo) {
-    this.taskNo = taskNo;
-  }
-
-  /**
-   * @return
-   */
-  public String getFactTimeStamp() {
-    return factTimeStamp;
-  }
-
-  /**
-   * @param factTimeStamp
-   */
-  public void setFactTimeStamp(String factTimeStamp) {
-    this.factTimeStamp = factTimeStamp;
-  }
-
-  public String getEscapeCharacter() {
-    return escapeCharacter;
-  }
-
-  public void setEscapeCharacter(String escapeCharacter) {
-    this.escapeCharacter = escapeCharacter;
-  }
-
-  public String getQuoteCharacter() { return quoteCharacter; }
-
-  public void setQuoteCharacter(String quoteCharacter) { this.quoteCharacter = quoteCharacter; }
-
-  public String getCommentCharacter() { return commentCharacter; }
-
-  public void setCommentCharacter(String commentCharacter) {
-    this.commentCharacter = commentCharacter;
-  }
-
-  public String getDateFormat() { return dateFormat; }
-
-  public void setDateFormat(String dateFormat) { this.dateFormat = dateFormat; }
-  /**
-   * @return
-   */
-  public String getMaxColumns() {
-    return maxColumns;
-  }
-
-  /**
-   * @param maxColumns
-   */
-  public void setMaxColumns(String maxColumns) {
-    this.maxColumns = maxColumns;
-  }
-
-  public String getRddIteratorKey() {
-    return rddIteratorKey;
-  }
-
-  public void setRddIteratorKey(String rddIteratorKey) {
-    this.rddIteratorKey = rddIteratorKey;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/SchemaInfo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/SchemaInfo.java b/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/SchemaInfo.java
deleted file mode 100644
index 88c4879..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/SchemaInfo.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.processing.api.dataloader;
-
-public class SchemaInfo {
-
-  /**
-   * databaseName
-   */
-  private String databaseName;
-
-  /**
-   * tableName
-   */
-  private String tableName;
-
-  /**
-   * isAutoAggregateRequest
-   */
-  private boolean isAutoAggregateRequest;
-
-  private String complexDelimiterLevel1;
-
-  private String complexDelimiterLevel2;
-  /**
-   * the value to be treated as null while data load
-   */
-  private String serializationNullFormat;
-
-  /**
-   * defines the string to specify whether the bad record logger should be enabled or not
-   */
-  private String badRecordsLoggerEnable;
-  /**
-   * defines the option to specify whether to bad record logger action
-   */
-  private String badRecordsLoggerAction;
-
-
-  public String getComplexDelimiterLevel1() {
-    return complexDelimiterLevel1;
-  }
-
-  public void setComplexDelimiterLevel1(String complexDelimiterLevel1) {
-    this.complexDelimiterLevel1 = complexDelimiterLevel1;
-  }
-
-  public String getComplexDelimiterLevel2() {
-    return complexDelimiterLevel2;
-  }
-
-  public void setComplexDelimiterLevel2(String complexDelimiterLevel2) {
-    this.complexDelimiterLevel2 = complexDelimiterLevel2;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  /**
-   * @return the isAutoAggregateRequest
-   */
-  public boolean isAutoAggregateRequest() {
-    return isAutoAggregateRequest;
-  }
-
-  /**
-   * @param isAutoAggregateRequest the isAutoAggregateRequest to set
-   */
-  public void setAutoAggregateRequest(boolean isAutoAggregateRequest) {
-    this.isAutoAggregateRequest = isAutoAggregateRequest;
-  }
-
-  /**
-   * @return the databaseName
-   */
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  /**
-   * @param databaseName the databaseName to set
-   */
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * the method returns the value to be treated as null while data load
-   * @return
-   */
-  public String getSerializationNullFormat() {
-    return serializationNullFormat;
-  }
-
-  /**
-   * the method sets the value to be treated as null while data load
-   * @param serializationNullFormat
-   */
-  public void setSerializationNullFormat(String serializationNullFormat) {
-    this.serializationNullFormat = serializationNullFormat;
-  }
-
-  /**
-   * returns the string to enable bad record logger
-   * @return
-   */
-  public String getBadRecordsLoggerEnable() {
-    return badRecordsLoggerEnable;
-  }
-
-  /**
-   * method sets the string to specify whether to enable or dissable the badrecord logger.
-   * @param badRecordsLoggerEnable
-   */
-  public void setBadRecordsLoggerEnable(String badRecordsLoggerEnable) {
-    this.badRecordsLoggerEnable = badRecordsLoggerEnable;
-  }
-
-  /**
-   * returns the option to set bad record logger action
-   * @return
-   */
-  public String getBadRecordsLoggerAction() {
-    return badRecordsLoggerAction;
-  }
-
-  /**
-   * set the option to set set bad record logger action
-   * @param badRecordsLoggerAction
-   */
-  public void setBadRecordsLoggerAction(String badRecordsLoggerAction) {
-    this.badRecordsLoggerAction = badRecordsLoggerAction;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/constants/DataProcessorConstants.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/constants/DataProcessorConstants.java b/processing/src/main/java/org/apache/carbondata/processing/constants/DataProcessorConstants.java
deleted file mode 100644
index 05f561f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/constants/DataProcessorConstants.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.constants;
-
-public final class DataProcessorConstants {
-  /**
-   *
-   */
-  public static final String CSV_DATALOADER = "CSV_DATALOADER";
-  /**
-   *
-   */
-  public static final String DATARESTRUCT = "DATARESTRUCT";
-  /**
-   * UPDATEMEMBER
-   */
-  public static final String UPDATEMEMBER = "UPDATEMEMBER";
-  /**
-   * number of days task should be in DB table
-   */
-  public static final String TASK_RETENTION_DAYS = "dataload.taskstatus.retention";
-  /**
-   * LOAD_FOLDER
-   */
-  public static final String LOAD_FOLDER = "Load_";
-  /**
-   * if bad record found
-   */
-  public static final long BAD_REC_FOUND = 223732673;
-  /**
-   * if bad record found
-   */
-  public static final long CSV_VALIDATION_ERRROR_CODE = 113732678;
-  /**
-   * Year Member val for data retention.
-   */
-  public static final String YEAR = "YEAR";
-
-  /**
-   * if data load fails due to bad record
-   */
-  public static final long BAD_REC_FAILURE_ERROR_CODE = 223732674;
-
-  private DataProcessorConstants() {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/constants/TableOptionConstant.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/constants/TableOptionConstant.java b/processing/src/main/java/org/apache/carbondata/processing/constants/TableOptionConstant.java
deleted file mode 100644
index 3917974..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/constants/TableOptionConstant.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.constants;
-
-/**
- * enum holds the value related to the ddl option
- */
-public enum TableOptionConstant {
-  SERIALIZATION_NULL_FORMAT("serialization_null_format"),
-  BAD_RECORDS_LOGGER_ENABLE("bad_records_logger_enable"),
-  BAD_RECORDS_ACTION("bad_records_action");
-
-  private String name;
-
-  /**
-   * constructor to initialize the enum value
-   * @param name
-   */
-  TableOptionConstant(String name) {
-    this.name = name;
-  }
-
-  public String getName() {
-    return name;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/csvload/BlockDetails.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/BlockDetails.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/BlockDetails.java
deleted file mode 100644
index d6d214b..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/BlockDetails.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.csvload;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-
-/**
- * blocks info
- */
-public class BlockDetails extends FileSplit implements Serializable {
-
-  /**
-   * serialization version
-   */
-  private static final long serialVersionUID = 2293906691860002339L;
-  //block offset
-  private long blockOffset;
-  //block length
-  private long blockLength;
-  //file path which block belong to
-  private String filePath;
-  // locations where this block exists
-  private String[] locations;
-
-  public BlockDetails(Path filePath, long blockOffset, long blockLength, String[] locations) {
-    super(filePath, blockOffset, blockLength, locations);
-    this.filePath = filePath.toString();
-    this.blockOffset = blockOffset;
-    this.blockLength = blockLength;
-    this.locations = locations;
-  }
-
-  public long getBlockOffset() {
-    return blockOffset;
-  }
-
-  public long getBlockLength() {
-    return blockLength;
-  }
-
-  public String getFilePath() {
-    return FileFactory.getUpdatedFilePath(filePath);
-  }
-
-  public void setFilePath(String filePath) {
-    this.filePath = filePath;
-  }
-
-  public String[] getLocations() {
-    return locations;
-  }
-
-  /** The file containing this split's data. */
-  @Override
-  public Path getPath() { return new Path(filePath); }
-
-  /** The position of the first byte in the file to process. */
-  @Override
-  public long getStart() { return blockOffset; }
-
-  /** The number of bytes in the file to process. */
-  @Override
-  public long getLength() { return blockLength; }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/csvload/BoundedInputStream.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/BoundedInputStream.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/BoundedInputStream.java
deleted file mode 100644
index 9f80c07..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/BoundedInputStream.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.csvload;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * Customarized reader class to read data from file
- * untill the upper threshold reached.
- */
-public class BoundedInputStream extends InputStream {
-
-  /**
-   * byte value of the new line character
-   */
-  private static final byte END_OF_LINE_BYTE_VALUE = '\n';
-
-  /**
-   * number of extra character to read
-   */
-  private static final int NUMBER_OF_EXTRA_CHARACTER_TO_READ = 100;
-
-  /**
-   * number of bytes remaining
-   */
-  private long remaining;
-  /**
-   * to check whether end of line is found
-   */
-  private boolean endOfLineFound = false;
-
-  private DataInputStream in;
-
-  public BoundedInputStream(DataInputStream in, long limit) {
-    this.in = in;
-    this.remaining = limit;
-  }
-
-  /**
-   * Below method will be used to read the data from file
-   *
-   * @throws IOException
-   *           problem while reading
-   */
-  @Override
-  public int read() throws IOException {
-    if (this.remaining == 0) {
-      return -1;
-    } else {
-      int var1 = this.in.read();
-      if (var1 >= 0) {
-        --this.remaining;
-      }
-
-      return var1;
-    }
-  }
-
-  /**
-   * Below method will be used to read the data from file. If limit reaches in
-   * that case it will read until new line character is reached
-   *
-   * @param buffer
-   *          buffer in which data will be read
-   * @param offset
-   *          from position to buffer will be filled
-   * @param length
-   *          number of character to be read
-   * @throws IOException
-   *           problem while reading
-   */
-  @Override
-  public int read(byte[] buffer, int offset, int length) throws IOException {
-    if (this.remaining == 0) {
-      return -1;
-    } else {
-      if (this.remaining < length) {
-        length = (int) this.remaining;
-      }
-
-      length = this.in.read(buffer, offset, length);
-      if (length >= 0) {
-        this.remaining -= length;
-        if (this.remaining == 0 && !endOfLineFound) {
-          endOfLineFound = true;
-          this.remaining += NUMBER_OF_EXTRA_CHARACTER_TO_READ;
-        } else if (endOfLineFound) {
-          int end = offset + length;
-          for (int i = offset; i < end; i++) {
-            if (buffer[i] == END_OF_LINE_BYTE_VALUE) {
-              this.remaining = 0;
-              return (i - offset) + 1;
-            }
-          }
-          this.remaining += NUMBER_OF_EXTRA_CHARACTER_TO_READ;
-        }
-      }
-      return length;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (in != null) {
-      in.close();
-    }
-  }
-
-  public long getRemaining() {
-    return  this.remaining;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.java
deleted file mode 100644
index c793126..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVInputFormat.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.processing.csvload;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.Reader;
-import java.nio.charset.Charset;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-import com.univocity.parsers.csv.CsvParser;
-import com.univocity.parsers.csv.CsvParserSettings;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.Seekable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.compress.CodecPool;
-import org.apache.hadoop.io.compress.CompressionCodec;
-import org.apache.hadoop.io.compress.CompressionCodecFactory;
-import org.apache.hadoop.io.compress.CompressionInputStream;
-import org.apache.hadoop.io.compress.Decompressor;
-import org.apache.hadoop.io.compress.SplitCompressionInputStream;
-import org.apache.hadoop.io.compress.SplittableCompressionCodec;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.util.LineReader;
-
-/**
- * An {@link org.apache.hadoop.mapreduce.InputFormat} for csv files.  Files are broken into lines.
- * Values are the line of csv files.
- */
-public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWritable> {
-
-  public static final String DELIMITER = "carbon.csvinputformat.delimiter";
-  public static final String DELIMITER_DEFAULT = ",";
-  public static final String COMMENT = "carbon.csvinputformat.comment";
-  public static final String COMMENT_DEFAULT = "#";
-  public static final String QUOTE = "carbon.csvinputformat.quote";
-  public static final String QUOTE_DEFAULT = "\"";
-  public static final String ESCAPE = "carbon.csvinputformat.escape";
-  public static final String ESCAPE_DEFAULT = "\\";
-  public static final String HEADER_PRESENT = "caron.csvinputformat.header.present";
-  public static final boolean HEADER_PRESENT_DEFAULT = false;
-  public static final String READ_BUFFER_SIZE = "carbon.csvinputformat.read.buffer.size";
-  public static final String READ_BUFFER_SIZE_DEFAULT = "65536";
-  public static final String MAX_COLUMNS = "carbon.csvinputformat.max.columns";
-  public static final String NUMBER_OF_COLUMNS = "carbon.csvinputformat.number.of.columns";
-  public static final int DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 2000;
-  public static final int THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 20000;
-
-  private static LogService LOGGER =
-      LogServiceFactory.getLogService(CSVInputFormat.class.toString());
-
-
-  @Override
-  public RecordReader<NullWritable, StringArrayWritable> createRecordReader(InputSplit inputSplit,
-      TaskAttemptContext context) throws IOException, InterruptedException {
-    return new CSVRecordReader();
-  }
-
-  @Override
-  protected boolean isSplitable(JobContext context, Path file) {
-    final CompressionCodec codec = new CompressionCodecFactory(context.getConfiguration())
-        .getCodec(file);
-    if (null == codec) {
-      return true;
-    }
-    return codec instanceof SplittableCompressionCodec;
-  }
-
-  /**
-   * Sets the comment char to configuration. Default it is #.
-   * @param configuration
-   * @param commentChar
-   */
-  public static void setCommentCharacter(Configuration configuration, String commentChar) {
-    if (commentChar != null && !commentChar.isEmpty()) {
-      configuration.set(COMMENT, commentChar);
-    }
-  }
-
-  /**
-   * Sets the delimiter to configuration. Default it is ','
-   * @param configuration
-   * @param delimiter
-   */
-  public static void setCSVDelimiter(Configuration configuration, String delimiter) {
-    if (delimiter != null && !delimiter.isEmpty()) {
-      configuration.set(DELIMITER, delimiter);
-    }
-  }
-
-  /**
-   * Sets the escape character to configuration. Default it is \
-   * @param configuration
-   * @param escapeCharacter
-   */
-  public static void setEscapeCharacter(Configuration configuration, String escapeCharacter) {
-    if (escapeCharacter != null && !escapeCharacter.isEmpty()) {
-      configuration.set(ESCAPE, escapeCharacter);
-    }
-  }
-
-  /**
-   * Whether header needs to read from csv or not. By default it is false.
-   * @param configuration
-   * @param headerExtractEnable
-   */
-  public static void setHeaderExtractionEnabled(Configuration configuration,
-      boolean headerExtractEnable) {
-    configuration.set(HEADER_PRESENT, String.valueOf(headerExtractEnable));
-  }
-
-  /**
-   * Sets the quote character to configuration. Default it is "
-   * @param configuration
-   * @param quoteCharacter
-   */
-  public static void setQuoteCharacter(Configuration configuration, String quoteCharacter) {
-    if (quoteCharacter != null && !quoteCharacter.isEmpty()) {
-      configuration.set(QUOTE, quoteCharacter);
-    }
-  }
-
-  /**
-   * Sets the read buffer size to configuration.
-   * @param configuration
-   * @param bufferSize
-   */
-  public static void setReadBufferSize(Configuration configuration, String bufferSize) {
-    if (bufferSize != null && !bufferSize.isEmpty()) {
-      configuration.set(READ_BUFFER_SIZE, bufferSize);
-    }
-  }
-
-  public static void setMaxColumns(Configuration configuration, String maxColumns) {
-    if (maxColumns != null) {
-      configuration.set(MAX_COLUMNS, maxColumns);
-    }
-  }
-
-  public static void setNumberOfColumns(Configuration configuration, String numberOfColumns) {
-    configuration.set(NUMBER_OF_COLUMNS, numberOfColumns);
-  }
-
-  /**
-   * Treats value as line in file. Key is null.
-   */
-  public static class CSVRecordReader extends RecordReader<NullWritable, StringArrayWritable> {
-
-    private long start;
-    private long end;
-    private BoundedInputStream boundedInputStream;
-    private Reader reader;
-    private CsvParser csvParser;
-    private StringArrayWritable value;
-    private String[] columns;
-    private Seekable filePosition;
-    private boolean isCompressedInput;
-    private Decompressor decompressor;
-
-    @Override
-    public void initialize(InputSplit inputSplit, TaskAttemptContext context)
-        throws IOException, InterruptedException {
-      FileSplit split = (FileSplit) inputSplit;
-      start = split.getStart();
-      end = start + split.getLength();
-      Path file = split.getPath();
-      Configuration job = context.getConfiguration();
-      CompressionCodec codec = (new CompressionCodecFactory(job)).getCodec(file);
-      FileSystem fs = file.getFileSystem(job);
-      int bufferSize = Integer.parseInt(job.get(READ_BUFFER_SIZE, READ_BUFFER_SIZE_DEFAULT));
-      FSDataInputStream fileIn = fs.open(file, bufferSize);
-      InputStream inputStream;
-      if (codec != null) {
-        isCompressedInput = true;
-        decompressor = CodecPool.getDecompressor(codec);
-        if (codec instanceof SplittableCompressionCodec) {
-          SplitCompressionInputStream scIn = ((SplittableCompressionCodec) codec)
-              .createInputStream(fileIn, decompressor, start, end, SplittableCompressionCodec
-                  .READ_MODE.BYBLOCK);
-          start = scIn.getAdjustedStart();
-          end = scIn.getAdjustedEnd();
-          if (start != 0) {
-            LineReader lineReader = new LineReader(scIn, 1);
-            start += lineReader.readLine(new Text(), 0);
-          }
-          filePosition = scIn;
-          inputStream = scIn;
-        } else {
-          CompressionInputStream cIn = codec.createInputStream(fileIn, decompressor);
-          filePosition = cIn;
-          inputStream = cIn;
-        }
-      } else {
-        fileIn.seek(start);
-        if (start != 0) {
-          LineReader lineReader = new LineReader(fileIn, 1);
-          start += lineReader.readLine(new Text(), 0);
-        }
-        boundedInputStream = new BoundedInputStream(fileIn, end - start);
-        filePosition = fileIn;
-        inputStream = boundedInputStream;
-      }
-      reader = new InputStreamReader(inputStream,
-          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-      csvParser = new CsvParser(extractCsvParserSettings(job));
-      csvParser.beginParsing(reader);
-    }
-
-    private CsvParserSettings extractCsvParserSettings(Configuration job) {
-      CsvParserSettings parserSettings = new CsvParserSettings();
-      parserSettings.getFormat().setDelimiter(job.get(DELIMITER, DELIMITER_DEFAULT).charAt(0));
-      parserSettings.getFormat().setComment(job.get(COMMENT, COMMENT_DEFAULT).charAt(0));
-      parserSettings.setLineSeparatorDetectionEnabled(true);
-      parserSettings.setNullValue("");
-      parserSettings.setEmptyValue("");
-      parserSettings.setIgnoreLeadingWhitespaces(false);
-      parserSettings.setIgnoreTrailingWhitespaces(false);
-      parserSettings.setSkipEmptyLines(false);
-      parserSettings.setMaxCharsPerColumn(100000);
-      String maxColumns = job.get(MAX_COLUMNS);
-      parserSettings.setMaxColumns(Integer.parseInt(maxColumns));
-      parserSettings.getFormat().setQuote(job.get(QUOTE, QUOTE_DEFAULT).charAt(0));
-      parserSettings.getFormat().setQuoteEscape(job.get(ESCAPE, ESCAPE_DEFAULT).charAt(0));
-      if (start == 0) {
-        parserSettings.setHeaderExtractionEnabled(job.getBoolean(HEADER_PRESENT,
-            HEADER_PRESENT_DEFAULT));
-      }
-      return parserSettings;
-    }
-
-    @Override
-    public boolean nextKeyValue() throws IOException, InterruptedException {
-      if (csvParser == null) {
-        return false;
-      }
-      columns = csvParser.parseNext();
-      if (columns == null) {
-        value = null;
-        return false;
-      }
-      if (value == null) {
-        value = new StringArrayWritable();
-      }
-      value.set(columns);
-      return true;
-    }
-
-    @Override
-    public NullWritable getCurrentKey() throws IOException, InterruptedException {
-      return NullWritable.get();
-    }
-
-    @Override
-    public StringArrayWritable getCurrentValue() throws IOException, InterruptedException {
-      return value;
-    }
-
-    private long getPos() throws IOException {
-      long retVal = start;
-      if (null != boundedInputStream) {
-        retVal = end - boundedInputStream.getRemaining();
-      } else if (isCompressedInput && null != filePosition) {
-        retVal = filePosition.getPos();
-      }
-      return retVal;
-    }
-
-    @Override
-    public float getProgress() throws IOException, InterruptedException {
-      return start == end ? 0.0F : Math.min(1.0F, (float) (getPos() -
-          start) / (float) (end - start));
-    }
-
-    @Override
-    public void close() throws IOException {
-      try {
-        if (reader != null) {
-          reader.close();
-        }
-        if (boundedInputStream != null) {
-          boundedInputStream.close();
-        }
-        if (null != csvParser) {
-          csvParser.stopParsing();
-        }
-      } finally {
-        reader = null;
-        boundedInputStream = null;
-        csvParser = null;
-        filePosition = null;
-        value = null;
-        if (decompressor != null) {
-          CodecPool.returnDecompressor(decompressor);
-          decompressor = null;
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVRecordReaderIterator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVRecordReaderIterator.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVRecordReaderIterator.java
deleted file mode 100644
index efe75ef..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/CSVRecordReaderIterator.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.csvload;
-
-import java.io.IOException;
-
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import com.univocity.parsers.common.TextParsingException;
-
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-/**
- * It is wrapper iterator around @{@link RecordReader}.
- */
-public class CSVRecordReaderIterator extends CarbonIterator<Object []> {
-
-  private RecordReader<NullWritable, StringArrayWritable> recordReader;
-
-  /**
-   * It is just a little hack to make recordreader as iterator. Usually we cannot call hasNext
-   * multiple times on record reader as it moves another line. To avoid that situation like hasNext
-   * only tells whether next row is present or not and next will move the pointer to next row after
-   * consuming it.
-   */
-  private boolean isConsumed;
-
-  private InputSplit split;
-
-  private TaskAttemptContext context;
-
-  public CSVRecordReaderIterator(RecordReader<NullWritable, StringArrayWritable> recordReader,
-      InputSplit split, TaskAttemptContext context) {
-    this.recordReader = recordReader;
-    this.split = split;
-    this.context = context;
-  }
-
-  @Override
-  public boolean hasNext() {
-    try {
-      if (!isConsumed) {
-        isConsumed = recordReader.nextKeyValue();
-        return isConsumed;
-      }
-      return true;
-    } catch (Exception e) {
-      if (e instanceof TextParsingException) {
-        throw new CarbonDataLoadingException(
-            CarbonDataProcessorUtil.trimErrorMessage(e.getMessage()));
-      }
-      throw new CarbonDataLoadingException(e);
-    }
-  }
-
-  @Override
-  public Object[] next() {
-    try {
-      String[] data = recordReader.getCurrentValue().get();
-      isConsumed = false;
-      return data;
-    } catch (Exception e) {
-      throw new CarbonDataLoadingException(e);
-    }
-  }
-
-  @Override
-  public void initialize() {
-    try {
-      recordReader.initialize(split, context);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  @Override
-  public void close() {
-    try {
-      recordReader.close();
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/csvload/StringArrayWritable.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/StringArrayWritable.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/StringArrayWritable.java
deleted file mode 100644
index 7eb3ec9..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/StringArrayWritable.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.csvload;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.charset.Charset;
-import java.util.Arrays;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * A String sequence that is usable as a key or value.
- */
-public class StringArrayWritable implements Writable {
-  private String[] values;
-
-  public String[] toStrings() {
-    return values;
-  }
-
-  public void set(String[] values) {
-    this.values = values;
-  }
-
-  public String[] get() {
-    return values;
-  }
-
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    int length = in.readInt();
-    values = new String[length];
-    for (int i = 0; i < length; i++) {
-      byte[] b = new byte[in.readInt()];
-      in.readFully(b);
-      values[i] = new String(b, Charset.defaultCharset());
-    }
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeInt(values.length);                 // write values
-    for (int i = 0; i < values.length; i++) {
-      byte[] b = values[i].getBytes(Charset.defaultCharset());
-      out.writeInt(b.length);
-      out.write(b);
-    }
-  }
-
-  @Override
-  public String toString() {
-    return Arrays.toString(values);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/manager/CarbonDataProcessorManager.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/manager/CarbonDataProcessorManager.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/manager/CarbonDataProcessorManager.java
deleted file mode 100644
index 5cf2078..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/manager/CarbonDataProcessorManager.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.dataprocessor.manager;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-public final class CarbonDataProcessorManager {
-  /**
-   * instance
-   */
-  private static final CarbonDataProcessorManager INSTANCE = new CarbonDataProcessorManager();
-
-  /**
-   * managerHandlerMap
-   */
-  private Map<String, Object> managerHandlerMap;
-
-  /**
-   * private constructor
-   */
-  private CarbonDataProcessorManager() {
-    managerHandlerMap = new HashMap<String, Object>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * Get instance method will be used to get the class instance
-   *
-   * @return
-   */
-  public static CarbonDataProcessorManager getInstance() {
-    return INSTANCE;
-  }
-
-  /**
-   * Below method will be used to get the lock object for all the data processing request.
-   * form the local map, if empty than it will update the map and return the lock object
-   *
-   * @param key
-   * @return
-   */
-  public synchronized Object getDataProcessingLockObject(String key) {
-    Object object = managerHandlerMap.get(key);
-    if (null == object) {
-      object = new Object();
-      managerHandlerMap.put(key, object);
-    }
-    return object;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
index 7661577..86a6744 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/ArrayDataType.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.processing.newflow.complexobjects.ArrayObject;
+import org.apache.carbondata.processing.loading.complexobjects.ArrayObject;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
index a9c2bfe..95d7d2e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/PrimitiveDataType.java
@@ -44,9 +44,9 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 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.processing.newflow.dictionary.DictionaryServerClientDictionary;
-import org.apache.carbondata.processing.newflow.dictionary.DirectDictionary;
-import org.apache.carbondata.processing.newflow.dictionary.PreCreatedDictionary;
+import org.apache.carbondata.processing.loading.dictionary.DictionaryServerClientDictionary;
+import org.apache.carbondata.processing.loading.dictionary.DirectDictionary;
+import org.apache.carbondata.processing.loading.dictionary.PreCreatedDictionary;
 
 /**
  * Primitive DataType stateless object used in data loading

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
index 68b6911..e90fd4a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/StructDataType.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.apache.carbondata.core.devapi.DictionaryGenerationException;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.processing.newflow.complexobjects.StructObject;
+import org.apache.carbondata.processing.loading.complexobjects.StructObject;
 
 /**
  * Struct DataType stateless object used in data loading

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/etl/DataLoadingException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/etl/DataLoadingException.java b/processing/src/main/java/org/apache/carbondata/processing/etl/DataLoadingException.java
deleted file mode 100644
index 4cb5961..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/etl/DataLoadingException.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.processing.etl;
-
-public class DataLoadingException extends Exception {
-  private static final long serialVersionUID = 1L;
-
-  private long errorCode = -1;
-
-  public DataLoadingException() {
-    super();
-  }
-
-  public DataLoadingException(long errorCode, String message) {
-    super(message);
-    this.errorCode = errorCode;
-  }
-
-  public DataLoadingException(String message) {
-    super(message);
-  }
-
-  public DataLoadingException(Throwable cause) {
-    super(cause);
-  }
-
-  public DataLoadingException(String message, Throwable cause) {
-    super(message, cause);
-  }
-
-  public long getErrorCode() {
-    return errorCode;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/exception/DataLoadingException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/exception/DataLoadingException.java b/processing/src/main/java/org/apache/carbondata/processing/exception/DataLoadingException.java
new file mode 100644
index 0000000..15ff95e
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/exception/DataLoadingException.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.processing.exception;
+
+public class DataLoadingException extends Exception {
+  private static final long serialVersionUID = 1L;
+
+  private long errorCode = -1;
+
+  public DataLoadingException() {
+    super();
+  }
+
+  public DataLoadingException(long errorCode, String message) {
+    super(message);
+    this.errorCode = errorCode;
+  }
+
+  public DataLoadingException(String message) {
+    super(message);
+  }
+
+  public DataLoadingException(Throwable cause) {
+    super(cause);
+  }
+
+  public DataLoadingException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  public long getErrorCode() {
+    return errorCode;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/exception/SliceMergerException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/exception/SliceMergerException.java b/processing/src/main/java/org/apache/carbondata/processing/exception/SliceMergerException.java
new file mode 100644
index 0000000..d9640a9
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/exception/SliceMergerException.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.exception;
+
+import java.util.Locale;
+
+public class SliceMergerException 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 SliceMergerException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param msg The error message for this exception.
+   */
+  public SliceMergerException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * This method is used to get the localized message.
+   *
+   * @param locale - A Locale object represents a specific geographical,
+   *               political, or cultural region.
+   * @return - Localized error message.
+   */
+  public String getLocalizedMessage(Locale locale) {
+    return "";
+  }
+
+  /**
+   * getLocalizedMessage
+   */
+  @Override public String getLocalizedMessage() {
+    return super.getLocalizedMessage();
+  }
+
+  /**
+   * getMessage
+   */
+  public String getMessage() {
+    return this.msg;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/AbstractDataLoadProcessorStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/AbstractDataLoadProcessorStep.java b/processing/src/main/java/org/apache/carbondata/processing/loading/AbstractDataLoadProcessorStep.java
new file mode 100644
index 0000000..9f2482b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/AbstractDataLoadProcessorStep.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading;
+
+import java.io.IOException;
+import java.util.Iterator;
+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.datastore.row.CarbonRow;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
+
+/**
+ * This base abstract class for data loading.
+ * It can do transformation jobs as per the implementation.
+ *
+ * Life cycle of this class is
+ * First initialize() is called to initialize the step
+ * then execute() is called to process the step logic and
+ * then close() is called to close any resources if any opened in the step.
+ */
+public abstract class AbstractDataLoadProcessorStep {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractDataLoadProcessorStep.class.getName());
+
+  protected CarbonDataLoadConfiguration configuration;
+
+  protected AbstractDataLoadProcessorStep child;
+
+  protected AtomicLong rowCounter;
+
+  protected boolean closed;
+
+  public AbstractDataLoadProcessorStep(CarbonDataLoadConfiguration configuration,
+      AbstractDataLoadProcessorStep child) {
+    this.configuration = configuration;
+    this.child = child;
+    this.rowCounter = new AtomicLong();
+    this.closed = false;
+  }
+
+  /**
+   * The output meta for this step. The data returns from this step is as per this meta.
+   *
+   */
+  public abstract DataField[] getOutput();
+
+  /**
+   * Initialization process for this step.
+   *
+   * @throws IOException
+   */
+  public void initialize() throws IOException {
+    if (LOGGER.isInfoEnabled()) {
+      // This thread prints the rows processed in each step for every 10 seconds.
+      new Thread() {
+        @Override public void run() {
+          while (!closed) {
+            try {
+              LOGGER.info("Rows processed in step " + getStepName() + " : " + rowCounter.get());
+              Thread.sleep(10000);
+            } catch (InterruptedException e) {
+              //ignore
+              LOGGER.error(e.getMessage());
+            }
+          }
+        }
+      }.start();
+    }
+  }
+
+  /**
+   * Tranform the data as per the implementation.
+   *
+   * @return Array of Iterator with data. It can be processed parallel if implementation class wants
+   * @throws CarbonDataLoadingException
+   */
+  public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
+    Iterator<CarbonRowBatch>[] childIters = child.execute();
+    Iterator<CarbonRowBatch>[] iterators = new Iterator[childIters.length];
+    for (int i = 0; i < childIters.length; i++) {
+      iterators[i] = getIterator(childIters[i]);
+    }
+    return iterators;
+  }
+
+  /**
+   * Create the iterator using child iterator.
+   *
+   * @param childIter
+   * @return new iterator with step specific processing.
+   */
+  protected Iterator<CarbonRowBatch> getIterator(final Iterator<CarbonRowBatch> childIter) {
+    return new CarbonIterator<CarbonRowBatch>() {
+      @Override public boolean hasNext() {
+        return childIter.hasNext();
+      }
+
+      @Override public CarbonRowBatch next() {
+        return processRowBatch(childIter.next());
+      }
+    };
+  }
+
+  /**
+   * Process the batch of rows as per the step logic.
+   *
+   * @param rowBatch
+   * @return processed row.
+   */
+  protected CarbonRowBatch processRowBatch(CarbonRowBatch rowBatch) {
+    CarbonRowBatch newBatch = new CarbonRowBatch(rowBatch.getSize());
+    while (rowBatch.hasNext()) {
+      newBatch.addRow(processRow(rowBatch.next()));
+    }
+    return newBatch;
+  }
+
+  /**
+   * Process the row as per the step logic.
+   *
+   * @param row
+   * @return processed row.
+   */
+  protected abstract CarbonRow processRow(CarbonRow row);
+
+  /**
+   * Get the step name for logging purpose.
+   * @return Step name
+   */
+  protected abstract String getStepName();
+
+
+  /**
+   * Close all resources.This method is called after execute() is finished.
+   * It will be called in both success and failure cases.
+   */
+  public void close() {
+    if (!closed) {
+      closed = true;
+      LOGGER.info("Total rows processed in step " + this.getStepName() + ": " + rowCounter.get());
+      if (child != null) {
+        child.close();
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/BadRecordsLogger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/BadRecordsLogger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/BadRecordsLogger.java
new file mode 100644
index 0000000..bc0ce3a
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/BadRecordsLogger.java
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+import java.io.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.HashMap;
+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.impl.FileFactory;
+import org.apache.carbondata.core.datastore.impl.FileFactory.FileType;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+
+public class BadRecordsLogger {
+
+  /**
+   * Comment for <code>LOGGER</code>
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(BadRecordsLogger.class.getName());
+  /**
+   * Which holds the key and if any bad rec found to check from API to update
+   * the status
+   */
+  private static Map<String, String> badRecordEntry =
+      new HashMap<String, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  /**
+   * File Name
+   */
+  private String fileName;
+  /**
+   * Store path
+   */
+  private String storePath;
+  /**
+   * FileChannel
+   */
+  private BufferedWriter bufferedWriter;
+  private DataOutputStream outStream;
+  /**
+   * csv file writer
+   */
+  private BufferedWriter bufferedCSVWriter;
+  private DataOutputStream outCSVStream;
+  /**
+   * bad record log file path
+   */
+  private String logFilePath;
+  /**
+   * csv file path
+   */
+  private String csvFilePath;
+
+  /**
+   * task key which is DatabaseName/TableName/tablename
+   */
+  private String taskKey;
+
+  private boolean badRecordsLogRedirect;
+
+  private boolean badRecordLoggerEnable;
+
+  private boolean badRecordConvertNullDisable;
+
+  private boolean isDataLoadFail;
+
+  // private final Object syncObject =new Object();
+
+  public BadRecordsLogger(String key, String fileName, String storePath,
+      boolean badRecordsLogRedirect, boolean badRecordLoggerEnable,
+      boolean badRecordConvertNullDisable, boolean isDataLoadFail) {
+    // Initially no bad rec
+    taskKey = key;
+    this.fileName = fileName;
+    this.storePath = storePath;
+    this.badRecordsLogRedirect = badRecordsLogRedirect;
+    this.badRecordLoggerEnable = badRecordLoggerEnable;
+    this.badRecordConvertNullDisable = badRecordConvertNullDisable;
+    this.isDataLoadFail = isDataLoadFail;
+  }
+
+  /**
+   * @param key DatabaseNaame/TableName/tablename
+   * @return return "Partially"
+   */
+  public static String hasBadRecord(String key) {
+    return badRecordEntry.get(key);
+  }
+
+  /**
+   * @param key DatabaseNaame/TableName/tablename
+   * @return remove key from the map
+   */
+  public static String removeBadRecordKey(String key) {
+    return badRecordEntry.remove(key);
+  }
+
+  public void addBadRecordsToBuilder(Object[] row, String reason)
+      throws CarbonDataLoadingException {
+    if (badRecordsLogRedirect || badRecordLoggerEnable) {
+      StringBuilder logStrings = new StringBuilder();
+      int size = row.length;
+      int count = size;
+      for (int i = 0; i < size; i++) {
+        if (null == row[i]) {
+          char ch =
+              logStrings.length() > 0 ? logStrings.charAt(logStrings.length() - 1) : (char) -1;
+          if (ch == ',') {
+            logStrings = logStrings.deleteCharAt(logStrings.lastIndexOf(","));
+          }
+          break;
+        } else if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(row[i].toString())) {
+          logStrings.append("null");
+        } else {
+          logStrings.append(row[i]);
+        }
+        if (count > 1) {
+          logStrings.append(',');
+        }
+        count--;
+      }
+      if (badRecordsLogRedirect) {
+        writeBadRecordsToCSVFile(logStrings);
+      }
+      if (badRecordLoggerEnable) {
+        logStrings.append("----->");
+        if (null != reason) {
+          if (reason.indexOf(CarbonCommonConstants.MEMBER_DEFAULT_VAL) > -1) {
+            logStrings
+                .append(reason.replace(CarbonCommonConstants.MEMBER_DEFAULT_VAL, "null"));
+          } else {
+            logStrings.append(reason);
+          }
+        }
+        writeBadRecordsToFile(logStrings);
+      }
+    } else {
+      // setting partial success entry since even if bad records are there then load
+      // status should be partial success regardless of bad record logged
+      badRecordEntry.put(taskKey, "Partially");
+    }
+  }
+
+  /**
+   *
+   */
+  private synchronized void writeBadRecordsToFile(StringBuilder logStrings)
+      throws CarbonDataLoadingException {
+    if (null == logFilePath) {
+      logFilePath =
+          this.storePath + File.separator + this.fileName + CarbonCommonConstants.LOG_FILE_EXTENSION
+              + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
+    }
+    try {
+      if (null == bufferedWriter) {
+        FileType fileType = FileFactory.getFileType(storePath);
+        if (!FileFactory.isFileExist(this.storePath, fileType)) {
+          // create the folders if not exist
+          FileFactory.mkdirs(this.storePath, fileType);
+
+          // create the files
+          FileFactory.createNewFile(logFilePath, fileType);
+        }
+
+        outStream = FileFactory.getDataOutputStream(logFilePath, fileType);
+
+        bufferedWriter = new BufferedWriter(new OutputStreamWriter(outStream,
+            Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      }
+      bufferedWriter.write(logStrings.toString());
+      bufferedWriter.newLine();
+    } catch (FileNotFoundException e) {
+      LOGGER.error("Bad Log Files not found");
+      throw new CarbonDataLoadingException("Bad Log Files not found", e);
+    } catch (IOException e) {
+      LOGGER.error("Error While writing bad record log File");
+      throw new CarbonDataLoadingException("Error While writing bad record log File", e);
+    } finally {
+      // if the Bad record file is created means it partially success
+      // if any entry present with key that means its have bad record for
+      // that key
+      badRecordEntry.put(taskKey, "Partially");
+    }
+  }
+
+  /**
+   * method will write the row having bad record in the csv file.
+   *
+   * @param logStrings
+   */
+  private synchronized void writeBadRecordsToCSVFile(StringBuilder logStrings)
+      throws CarbonDataLoadingException {
+    if (null == csvFilePath) {
+      csvFilePath =
+          this.storePath + File.separator + this.fileName + CarbonCommonConstants.CSV_FILE_EXTENSION
+              + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
+    }
+    try {
+      if (null == bufferedCSVWriter) {
+        FileType fileType = FileFactory.getFileType(storePath);
+        if (!FileFactory.isFileExist(this.storePath, fileType)) {
+          // create the folders if not exist
+          FileFactory.mkdirs(this.storePath, fileType);
+
+          // create the files
+          FileFactory.createNewFile(csvFilePath, fileType);
+        }
+
+        outCSVStream = FileFactory.getDataOutputStream(csvFilePath, fileType);
+
+        bufferedCSVWriter = new BufferedWriter(new OutputStreamWriter(outCSVStream,
+            Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      }
+      bufferedCSVWriter.write(logStrings.toString());
+      bufferedCSVWriter.newLine();
+    } catch (FileNotFoundException e) {
+      LOGGER.error("Bad record csv Files not found");
+      throw new CarbonDataLoadingException("Bad record csv Files not found", e);
+    } catch (IOException e) {
+      LOGGER.error("Error While writing bad record csv File");
+      throw new CarbonDataLoadingException("Error While writing bad record csv File", e);
+    }
+    finally {
+      badRecordEntry.put(taskKey, "Partially");
+    }
+  }
+
+  public boolean isBadRecordConvertNullDisable() {
+    return badRecordConvertNullDisable;
+  }
+
+  public boolean isDataLoadFail() {
+    return isDataLoadFail;
+  }
+
+  public boolean isBadRecordLoggerEnable() {
+    return badRecordLoggerEnable;
+  }
+
+  public boolean isBadRecordsLogRedirect() {
+    return badRecordsLogRedirect;
+  }
+
+  /**
+   * closeStreams void
+   */
+  public synchronized void closeStreams() {
+    CarbonUtil.closeStreams(bufferedWriter, outStream, bufferedCSVWriter, outCSVStream);
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..7309c91
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/CarbonDataLoadConfiguration.java
@@ -0,0 +1,313 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.datastore.TableSpec;
+import org.apache.carbondata.core.keygenerator.KeyGenerator;
+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.table.column.CarbonColumn;
+import org.apache.carbondata.processing.loading.converter.DictionaryCardinalityFinder;
+
+public class CarbonDataLoadConfiguration {
+
+  private DataField[] dataFields;
+
+  private AbsoluteTableIdentifier tableIdentifier;
+
+  private String[] header;
+
+  private String partitionId;
+
+  private String segmentId;
+
+  private String taskNo;
+
+  private BucketingInfo bucketingInfo;
+
+  private Map<String, Object> dataLoadProperties = new HashMap<>();
+
+  /**
+   *  Use one pass to generate dictionary
+   */
+  private boolean useOnePass;
+
+  /**
+   * dictionary server host
+   */
+  private String dictionaryServerHost;
+
+  /**
+   * dictionary sever port
+   */
+  private int dictionaryServerPort;
+
+  private boolean preFetch;
+
+  private int dimensionCount;
+
+  private int measureCount;
+
+  private int noDictionaryCount;
+
+  private int complexColumnCount;
+
+  /**
+   * schema updated time stamp to be used for restructure scenarios
+   */
+  private long schemaUpdatedTimeStamp;
+
+  private DictionaryCardinalityFinder cardinalityFinder;
+
+  private int numberOfSortColumns;
+
+  private int numberOfNoDictSortColumns;
+
+  // contains metadata used in write step of loading process
+  private TableSpec tableSpec;
+
+  public CarbonDataLoadConfiguration() {
+  }
+
+  public void setDataFields(DataField[] dataFields) {
+    this.dataFields = dataFields;
+
+    // set counts for each column category
+    for (DataField dataField : dataFields) {
+      CarbonColumn column = dataField.getColumn();
+      if (column.isDimension()) {
+        dimensionCount++;
+        if (!dataField.hasDictionaryEncoding()) {
+          noDictionaryCount++;
+        }
+      }
+      if (column.isComplex()) {
+        complexColumnCount++;
+      }
+      if (column.isMeasure()) {
+        measureCount++;
+      }
+    }
+  }
+
+  public DataField[] getDataFields() {
+    return dataFields;
+  }
+
+  public int getDimensionCount() {
+    return dimensionCount;
+  }
+
+  public int getNoDictionaryCount() {
+    return noDictionaryCount;
+  }
+
+  public int getComplexColumnCount() {
+    return complexColumnCount;
+  }
+
+  public int getMeasureCount() {
+    return measureCount;
+  }
+
+  public void setNumberOfSortColumns(int numberOfSortColumns) {
+    this.numberOfSortColumns = numberOfSortColumns;
+  }
+
+  public int getNumberOfSortColumns() {
+    return this.numberOfSortColumns;
+  }
+
+  public boolean isSortTable() {
+    return this.numberOfSortColumns > 0;
+  }
+
+  public void setNumberOfNoDictSortColumns(int numberOfNoDictSortColumns) {
+    this.numberOfNoDictSortColumns = numberOfNoDictSortColumns;
+  }
+
+  public int getNumberOfNoDictSortColumns() {
+    return this.numberOfNoDictSortColumns;
+  }
+
+  public String[] getHeader() {
+    return header;
+  }
+
+  public void setHeader(String[] header) {
+    this.header = header;
+  }
+
+  public AbsoluteTableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public void setTableIdentifier(AbsoluteTableIdentifier tableIdentifier) {
+    this.tableIdentifier = tableIdentifier;
+  }
+
+  public String getPartitionId() {
+    return partitionId;
+  }
+
+  public void setPartitionId(String partitionId) {
+    this.partitionId = partitionId;
+  }
+
+  public String getSegmentId() {
+    return segmentId;
+  }
+
+  public void setSegmentId(String segmentId) {
+    this.segmentId = segmentId;
+  }
+
+  public String getTaskNo() {
+    return taskNo;
+  }
+
+  public void setTaskNo(String taskNo) {
+    this.taskNo = taskNo;
+  }
+
+  public void setDataLoadProperty(String key, Object value) {
+    dataLoadProperties.put(key, value);
+  }
+
+  public Object getDataLoadProperty(String key) {
+    return dataLoadProperties.get(key);
+  }
+
+  public BucketingInfo getBucketingInfo() {
+    return bucketingInfo;
+  }
+
+  public void setBucketingInfo(BucketingInfo bucketingInfo) {
+    this.bucketingInfo = bucketingInfo;
+  }
+
+  public boolean getUseOnePass() {
+    return useOnePass;
+  }
+
+  public void setUseOnePass(boolean useOnePass) {
+    this.useOnePass = useOnePass;
+  }
+
+  public String getDictionaryServerHost() {
+    return dictionaryServerHost;
+  }
+
+  public void setDictionaryServerHost(String dictionaryServerHost) {
+    this.dictionaryServerHost = dictionaryServerHost;
+  }
+
+  public int getDictionaryServerPort() {
+    return dictionaryServerPort;
+  }
+
+  public void setDictionaryServerPort(int dictionaryServerPort) {
+    this.dictionaryServerPort = dictionaryServerPort;
+  }
+
+  public boolean isPreFetch() {
+    return preFetch;
+  }
+
+  public void setPreFetch(boolean preFetch) {
+    this.preFetch = preFetch;
+  }
+
+  public long getSchemaUpdatedTimeStamp() {
+    return schemaUpdatedTimeStamp;
+  }
+
+  public void setSchemaUpdatedTimeStamp(long schemaUpdatedTimeStamp) {
+    this.schemaUpdatedTimeStamp = schemaUpdatedTimeStamp;
+  }
+
+  public DictionaryCardinalityFinder getCardinalityFinder() {
+    return cardinalityFinder;
+  }
+
+  public void setCardinalityFinder(DictionaryCardinalityFinder cardinalityFinder) {
+    this.cardinalityFinder = cardinalityFinder;
+  }
+
+  public DataType[] getMeasureDataType() {
+    List<Integer> measureIndexes = new ArrayList<>(dataFields.length);
+    int measureCount = 0;
+    for (int i = 0; i < dataFields.length; i++) {
+      if (!dataFields[i].getColumn().isDimension()) {
+        measureIndexes.add(i);
+        measureCount++;
+      }
+    }
+
+    DataType[] type = new DataType[measureCount];
+    for (int i = 0; i < type.length; i++) {
+      type[i] = dataFields[measureIndexes.get(i)].getColumn().getDataType();
+    }
+    return type;
+  }
+
+  public int[] calcDimensionLengths() {
+    int[] dimLensWithComplex = getCardinalityFinder().getCardinality();
+    if (!isSortTable()) {
+      for (int i = 0; i < dimLensWithComplex.length; i++) {
+        if (dimLensWithComplex[i] != 0) {
+          dimLensWithComplex[i] = Integer.MAX_VALUE;
+        }
+      }
+    }
+    List<Integer> dimsLenList = new ArrayList<Integer>();
+    for (int eachDimLen : dimLensWithComplex) {
+      if (eachDimLen != 0) dimsLenList.add(eachDimLen);
+    }
+    int[] dimLens = new int[dimsLenList.size()];
+    for (int i = 0; i < dimsLenList.size(); i++) {
+      dimLens[i] = dimsLenList.get(i);
+    }
+    return dimLens;
+  }
+
+  public KeyGenerator[] createKeyGeneratorForComplexDimension() {
+    int[] dimLens = calcDimensionLengths();
+    KeyGenerator[] complexKeyGenerators = new KeyGenerator[dimLens.length];
+    for (int i = 0; i < dimLens.length; i++) {
+      complexKeyGenerators[i] =
+          KeyGeneratorFactory.getKeyGenerator(new int[] { dimLens[i] });
+    }
+    return complexKeyGenerators;
+  }
+
+  public TableSpec getTableSpec() {
+    return tableSpec;
+  }
+
+  public void setTableSpec(TableSpec tableSpec) {
+    this.tableSpec = tableSpec;
+  }
+}


[15/50] [abbrv] carbondata git commit: [CARBONDATA-1486] Fixed issue of table status updation on insert overwrite failure and exception thrown while deletion of stale folders

Posted by ja...@apache.org.
[CARBONDATA-1486] Fixed issue of table status updation on insert overwrite failure and exception thrown while deletion of stale folders

This closes #1368


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

Branch: refs/heads/streaming_ingest
Commit: c4f312fa1dda5774fac715aa96fde3094f42a592
Parents: d304ef9
Author: manishgupta88 <to...@gmail.com>
Authored: Mon Sep 18 18:03:04 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Sep 28 01:47:56 2017 +0800

----------------------------------------------------------------------
 .../carbondata/spark/load/CarbonLoaderUtil.java | 23 +++++++++++++++++--
 .../load/DataLoadProcessorStepOnSpark.scala     |  2 +-
 .../carbondata/spark/util/CommonUtil.scala      | 24 ++++++++++++++++++++
 .../spark/rdd/CarbonDataRDDFactory.scala        |  9 +++++---
 .../newflow/sort/SortStepRowUtil.java           |  3 +--
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  2 +-
 6 files changed, 54 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c4f312fa/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
index 7c2d157..9fe003f 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
@@ -289,9 +289,18 @@ public final class CarbonLoaderUtil {
           newMetaEntry.setLoadName(segmentId);
           loadModel.setLoadMetadataDetails(listOfLoadFolderDetails);
           loadModel.setSegmentId(segmentId);
+          // Exception should be thrown if:
+          // 1. If insert overwrite is in progress and any other load or insert operation
+          // is triggered
+          // 2. If load or insert into operation is in progress and insert overwrite operation
+          // is triggered
           for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
             if (entry.getLoadStatus().equals(LoadStatusType.INSERT_OVERWRITE.getMessage())) {
               throw new RuntimeException("Already insert overwrite is in progress");
+            } else if (
+                newMetaEntry.getLoadStatus().equals(LoadStatusType.INSERT_OVERWRITE.getMessage())
+                    && entry.getLoadStatus().equals(LoadStatusType.IN_PROGRESS.getMessage())) {
+              throw new RuntimeException("Already insert into or load is in progress");
             }
           }
           listOfLoadFolderDetails.add(newMetaEntry);
@@ -318,7 +327,11 @@ public final class CarbonLoaderUtil {
                 // For insert overwrite, we will delete the old segment folder immediately
                 // So collect the old segments here
                 String path = carbonTablePath.getCarbonDataDirectoryPath("0", entry.getLoadName());
-                staleFolders.add(FileFactory.getCarbonFile(path));
+                // 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))) {
+                  staleFolders.add(FileFactory.getCarbonFile(path));
+                }
               }
             }
           }
@@ -328,7 +341,13 @@ public final class CarbonLoaderUtil {
             .toArray(new LoadMetadataDetails[listOfLoadFolderDetails.size()]));
         // Delete all old stale segment folders
         for (CarbonFile staleFolder : staleFolders) {
-          CarbonUtil.deleteFoldersAndFiles(staleFolder);
+          // try block is inside for loop because even if there is failure in deletion of 1 stale
+          // folder still remaining stale folders should be deleted
+          try {
+            CarbonUtil.deleteFoldersAndFiles(staleFolder);
+          } catch (IOException | InterruptedException e) {
+            LOGGER.error("Failed to delete stale folder: " + e.getMessage());
+          }
         }
         status = true;
       } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c4f312fa/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 aaf7926..6943dcb 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
@@ -136,7 +136,7 @@ object DataLoadProcessorStepOnSpark {
 
       override def next(): CarbonRow = {
         val row =
-          new CarbonRow(SortStepRowUtil.convertRow(rows.next().getData, sortParameters, true))
+          new CarbonRow(SortStepRowUtil.convertRow(rows.next().getData, sortParameters))
         rowCounter.add(1)
         row
       }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c4f312fa/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 f123624..5040e69 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
@@ -537,6 +537,30 @@ object CommonUtil {
     }
   }
 
+  /**
+   * This method will update the load failure entry in the table status file
+   *
+   * @param model
+   */
+  def updateTableStatusForFailure(
+      model: CarbonLoadModel): Unit = {
+    // in case if failure the load status should be "Marked for delete" so that it will be taken
+    // care during clean up
+    val loadStatus = CarbonCommonConstants.MARKED_FOR_DELETE
+    // always the last entry in the load metadata details will be the current load entry
+    val loadMetaEntry = model.getLoadMetadataDetails.get(model.getLoadMetadataDetails.size - 1)
+    CarbonLoaderUtil
+      .populateNewLoadMetaEntry(loadMetaEntry, loadStatus, model.getFactTimeStamp, true)
+    val updationStatus = CarbonLoaderUtil.recordLoadMetadata(loadMetaEntry, model, false, false)
+    if (!updationStatus) {
+      sys
+        .error(s"Failed to update failure entry in table status for ${
+          model
+            .getDatabaseName
+        }.${ model.getTableName }")
+    }
+  }
+
   def readLoadMetadataDetails(model: CarbonLoadModel): Unit = {
     val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetaDataFilepath
     val details = SegmentStatusManager.readLoadMetadata(metadataPath)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c4f312fa/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 0edfccf..3d67a79 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
@@ -1069,13 +1069,14 @@ object CarbonDataRDDFactory {
         return
       }
       if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_FAILURE) {
+        // update the load entry in table status file for changing the status to failure
+        CommonUtil.updateTableStatusForFailure(carbonLoadModel)
         LOGGER.info("********starting clean up**********")
         CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
         LOGGER.info("********clean up done**********")
         LOGGER.audit(s"Data load is failed for " +
             s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
         LOGGER.warn("Cannot write load metadata file as data load failed")
-        updateStatus(status, loadStatus)
         throw new Exception(errorMessage)
       } else {
         // check if data load fails due to bad record and throw data load failure due to
@@ -1083,16 +1084,19 @@ object CarbonDataRDDFactory {
         if (loadStatus == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS &&
             status(0)._2._2.failureCauses == FailureCauses.BAD_RECORDS &&
             carbonLoadModel.getBadRecordsAction.split(",")(1) == LoggerAction.FAIL.name) {
+          // update the load entry in table status file for changing the status to failure
+          CommonUtil.updateTableStatusForFailure(carbonLoadModel)
           LOGGER.info("********starting clean up**********")
           CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
           LOGGER.info("********clean up done**********")
           LOGGER.audit(s"Data load is failed for " +
                        s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-          updateStatus(status, CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)
           throw new Exception(status(0)._2._2.errorMsg)
         }
         // if segment is empty then fail the data load
         if (!CarbonLoaderUtil.isValidSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)) {
+          // update the load entry in table status file for changing the status to failure
+          CommonUtil.updateTableStatusForFailure(carbonLoadModel)
           LOGGER.info("********starting clean up**********")
           CarbonLoaderUtil.deleteSegment(carbonLoadModel, carbonLoadModel.getSegmentId.toInt)
           LOGGER.info("********clean up done**********")
@@ -1100,7 +1104,6 @@ object CarbonDataRDDFactory {
                        s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }" +
                        " as there is no data to load")
           LOGGER.warn("Cannot write load metadata file as data load failed")
-          updateStatus(status, CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)
           throw new Exception("No Data to load")
         }
         writeDictionary(carbonLoadModel, result, false)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c4f312fa/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java
index 5238c3c..62434bc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java
@@ -21,8 +21,7 @@ import org.apache.carbondata.core.util.NonDictionaryUtil;
 import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
 
 public class SortStepRowUtil {
-  public static Object[] convertRow(Object[] data, SortParameters parameters,
-      boolean needConvertDecimalToByte) {
+  public static Object[] convertRow(Object[] data, SortParameters parameters) {
     int measureCount = parameters.getMeasureColCount();
     int dimensionCount = parameters.getDimColCount();
     int complexDimensionCount = parameters.getComplexDimColCount();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c4f312fa/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
index 1455365..e3bbdcb 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
@@ -183,7 +183,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
    * @return sorted row
    */
   public Object[] next() {
-    return SortStepRowUtil.convertRow(getSortedRecordFromFile(), parameters, false);
+    return SortStepRowUtil.convertRow(getSortedRecordFromFile(), parameters);
   }
 
   /**


[35/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..14ab838
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeCarbonRowPage.java
@@ -0,0 +1,405 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+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.core.metadata.datatype.DataType;
+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;
+
+  private long sizeToBeUsed;
+
+  private MemoryBlock dataBlock;
+
+  private boolean saveToDisk;
+
+  private MemoryManagerType managerType;
+
+  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;
+    this.saveToDisk = saveToDisk;
+    this.nullSetWords = new long[((measureSize - 1) >> 6) + 1];
+    this.taskId = taskId;
+    buffer = new IntPointerBuffer(this.taskId);
+    this.dataBlock = memoryBlock;
+    // TODO Only using 98% of space for safe side.May be we can have different logic.
+    sizeToBeUsed = dataBlock.size() - (dataBlock.size() * 5) / 100;
+    this.managerType = MemoryManagerType.UNSAFE_MEMORY_MANAGER;
+  }
+
+  public int addRow(Object[] row) {
+    int size = addRow(row, dataBlock.getBaseOffset() + lastSize);
+    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) {
+        switch (measureDataType[mesCount]) {
+          case SHORT:
+            Short sval = (Short) value;
+            CarbonUnsafe.getUnsafe().putShort(baseObject, address + size, sval);
+            size += 2;
+            break;
+          case INT:
+            Integer ival = (Integer) value;
+            CarbonUnsafe.getUnsafe().putInt(baseObject, address + size, ival);
+            size += 4;
+            break;
+          case LONG:
+            Long val = (Long) value;
+            CarbonUnsafe.getUnsafe().putLong(baseObject, address + size, val);
+            size += 8;
+            break;
+          case DOUBLE:
+            Double doubleVal = (Double) value;
+            CarbonUnsafe.getUnsafe().putDouble(baseObject, address + size, doubleVal);
+            size += 8;
+            break;
+          case DECIMAL:
+            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;
+            break;
+          default:
+            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;
+  }
+
+  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)) {
+        switch (measureDataType[mesCount]) {
+          case SHORT:
+            Short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+            size += 2;
+            rowToFill[dimensionSize + mesCount] = sval;
+            break;
+          case INT:
+            Integer ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+            size += 4;
+            rowToFill[dimensionSize + mesCount] = ival;
+            break;
+          case LONG:
+            Long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
+            size += 8;
+            rowToFill[dimensionSize + mesCount] = val;
+            break;
+          case DOUBLE:
+            Double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
+            size += 8;
+            rowToFill[dimensionSize + mesCount] = doubleVal;
+            break;
+          case DECIMAL:
+            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);
+            break;
+          default:
+            throw new IllegalArgumentException("unsupported data type:" +
+                measureDataType[mesCount]);
+        }
+      } else {
+        rowToFill[dimensionSize + mesCount] = null;
+      }
+    }
+    return rowToFill;
+  }
+
+  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)) {
+        switch (measureDataType[mesCount]) {
+          case SHORT:
+            short sval = CarbonUnsafe.getUnsafe().getShort(baseObject, address + size);
+            size += 2;
+            stream.writeShort(sval);
+            break;
+          case INT:
+            int ival = CarbonUnsafe.getUnsafe().getInt(baseObject, address + size);
+            size += 4;
+            stream.writeInt(ival);
+            break;
+          case LONG:
+            long val = CarbonUnsafe.getUnsafe().getLong(baseObject, address + size);
+            size += 8;
+            stream.writeLong(val);
+            break;
+          case DOUBLE:
+            double doubleVal = CarbonUnsafe.getUnsafe().getDouble(baseObject, address + size);
+            size += 8;
+            stream.writeDouble(doubleVal);
+            break;
+          case DECIMAL:
+            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);
+            break;
+          default:
+            throw new IllegalArgumentException("unsupported data type:" +
+                measureDataType[mesCount]);
+        }
+      }
+    }
+  }
+
+  public void freeMemory() {
+    switch (managerType) {
+      case UNSAFE_MEMORY_MANAGER:
+        UnsafeMemoryManager.INSTANCE.freeMemory(taskId, dataBlock);
+        break;
+      default:
+        UnsafeSortMemoryManager.INSTANCE.freeMemory(taskId, dataBlock);
+        buffer.freeMemory();
+    }
+  }
+
+  public boolean isSaveToDisk() {
+    return saveToDisk;
+  }
+
+  public IntPointerBuffer getBuffer() {
+    return buffer;
+  }
+
+  public int getUsedSize() {
+    return lastSize;
+  }
+
+  public boolean canAdd() {
+    return lastSize < sizeToBeUsed;
+  }
+
+  public MemoryBlock getDataBlock() {
+    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 void setNewDataBlock(MemoryBlock newMemoryBlock) {
+    this.dataBlock = newMemoryBlock;
+    this.managerType = MemoryManagerType.UNSAFE_SORT_MEMORY_MANAGER;
+  }
+
+  public enum MemoryManagerType {
+    UNSAFE_MEMORY_MANAGER, UNSAFE_SORT_MEMORY_MANAGER
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..88b72aa
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/UnsafeSortDataRows.java
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+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.memory.CarbonUnsafe;
+import org.apache.carbondata.core.memory.IntPointerBuffer;
+import org.apache.carbondata.core.memory.MemoryBlock;
+import org.apache.carbondata.core.memory.MemoryException;
+import org.apache.carbondata.core.memory.UnsafeMemoryManager;
+import org.apache.carbondata.core.memory.UnsafeSortMemoryManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+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.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.util.CarbonDataProcessorUtil;
+
+public class UnsafeSortDataRows {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeSortDataRows.class.getName());
+  /**
+   * threadStatusObserver
+   */
+  private ThreadStatusObserver threadStatusObserver;
+  /**
+   * executor service for data sort holder
+   */
+  private ExecutorService dataSorterAndWriterExecutorService;
+  /**
+   * semaphore which will used for managing sorted data object arrays
+   */
+
+  private SortParameters parameters;
+
+  private UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger;
+
+  private UnsafeCarbonRowPage rowPage;
+
+  private final Object addRowsLock = new Object();
+
+  private long inMemoryChunkSize;
+
+  private boolean enableInMemoryIntermediateMerge;
+
+  private int bytesAdded;
+
+  private long maxSizeAllowed;
+
+  /**
+   * semaphore which will used for managing sorted data object arrays
+   */
+  private Semaphore semaphore;
+
+  private final long taskId;
+
+  public UnsafeSortDataRows(SortParameters parameters,
+      UnsafeIntermediateMerger unsafeInMemoryIntermediateFileMerger, int inMemoryChunkSize) {
+    this.parameters = parameters;
+
+    this.unsafeInMemoryIntermediateFileMerger = unsafeInMemoryIntermediateFileMerger;
+
+    // observer of writing file in thread
+    this.threadStatusObserver = new ThreadStatusObserver();
+    this.taskId = ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId();
+    this.inMemoryChunkSize = inMemoryChunkSize;
+    this.inMemoryChunkSize = inMemoryChunkSize * 1024L * 1024L;
+    enableInMemoryIntermediateMerge = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.ENABLE_INMEMORY_MERGE_SORT,
+            CarbonCommonConstants.ENABLE_INMEMORY_MERGE_SORT_DEFAULT));
+
+    this.maxSizeAllowed = parameters.getBatchSortSizeinMb();
+    if (maxSizeAllowed <= 0) {
+      // If user does not input any memory size, then take half the size of usable memory configured
+      // in sort memory size.
+      this.maxSizeAllowed = UnsafeMemoryManager.INSTANCE.getUsableMemory() / 2;
+    } else {
+      this.maxSizeAllowed = this.maxSizeAllowed * 1024 * 1024;
+    }
+  }
+
+  /**
+   * This method will be used to initialize
+   */
+  public void initialize() throws MemoryException {
+    MemoryBlock baseBlock =
+        UnsafeMemoryManager.allocateMemoryWithRetry(this.taskId, inMemoryChunkSize);
+    boolean isMemoryAvailable =
+        UnsafeSortMemoryManager.INSTANCE.isMemoryAvailable(baseBlock.size());
+    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);
+    // Delete if any older file exists in sort temp folder
+    deleteSortLocationIfExists();
+
+    // create new sort temp directory
+    CarbonDataProcessorUtil.createLocations(parameters.getTempFileLocation());
+    this.dataSorterAndWriterExecutorService =
+        Executors.newFixedThreadPool(parameters.getNumberOfCores());
+    semaphore = new Semaphore(parameters.getNumberOfCores());
+  }
+
+  public boolean canAdd() {
+    return bytesAdded < maxSizeAllowed;
+  }
+
+  /**
+   * This method will be used to add new row
+   *
+   * @param rowBatch new rowBatch
+   * @throws CarbonSortKeyAndGroupByException problem while writing
+   */
+  public void addRowBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
+    // 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
+    synchronized (addRowsLock) {
+      addBatch(rowBatch, size);
+    }
+  }
+
+  /**
+   * This method will be used to add new row
+   *
+   * @param rowBatch new rowBatch
+   * @param size
+   * @throws CarbonSortKeyAndGroupByException problem while writing
+   */
+  public void addRowBatchWithOutSync(Object[][] rowBatch, int size)
+      throws CarbonSortKeyAndGroupByException {
+    // 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
+    addBatch(rowBatch, size);
+  }
+
+  private void addBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
+    for (int i = 0; i < size; i++) {
+      if (rowPage.canAdd()) {
+        bytesAdded += rowPage.addRow(rowBatch[i]);
+      } else {
+        try {
+          if (enableInMemoryIntermediateMerge) {
+            unsafeInMemoryIntermediateFileMerger.startInmemoryMergingIfPossible();
+          }
+          unsafeInMemoryIntermediateFileMerger.startFileMergingIfPossible();
+          semaphore.acquire();
+          dataSorterAndWriterExecutorService.execute(new DataSorterAndWriter(rowPage));
+          MemoryBlock memoryBlock =
+              UnsafeMemoryManager.allocateMemoryWithRetry(this.taskId, inMemoryChunkSize);
+          boolean saveToDisk =
+              UnsafeSortMemoryManager.INSTANCE.isMemoryAvailable(memoryBlock.size());
+          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]);
+        } catch (Exception e) {
+          LOGGER.error(
+                  "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
+          throw new CarbonSortKeyAndGroupByException(e);
+        }
+
+      }
+    }
+  }
+
+  /**
+   * This method will be used to add new row
+   */
+  public void addRow(Object[] row) throws CarbonSortKeyAndGroupByException {
+    // 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);
+    } else {
+      try {
+        if (enableInMemoryIntermediateMerge) {
+          unsafeInMemoryIntermediateFileMerger.startInmemoryMergingIfPossible();
+        }
+        unsafeInMemoryIntermediateFileMerger.startFileMergingIfPossible();
+        semaphore.acquire();
+        dataSorterAndWriterExecutorService.submit(new DataSorterAndWriter(rowPage));
+        MemoryBlock memoryBlock =
+            UnsafeMemoryManager.allocateMemoryWithRetry(this.taskId, inMemoryChunkSize);
+        boolean saveToDisk = UnsafeSortMemoryManager.INSTANCE.isMemoryAvailable(memoryBlock.size());
+        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);
+      } catch (Exception e) {
+        LOGGER.error(
+            "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
+        throw new CarbonSortKeyAndGroupByException(e);
+      }
+
+    }
+  }
+
+  /**
+   * Below method will be used to start storing process This method will get
+   * all the temp files present in sort temp folder then it will create the
+   * record holder heap and then it will read first record from each file and
+   * initialize the heap
+   *
+   * @throws InterruptedException
+   */
+  public void startSorting() throws InterruptedException {
+    LOGGER.info("Unsafe based sorting will be used");
+    if (this.rowPage.getUsedSize() > 0) {
+      TimSort<UnsafeCarbonRow, IntPointerBuffer> timSort = new TimSort<>(
+          new UnsafeIntSortDataFormat(rowPage));
+      if (parameters.getNumberOfNoDictSortColumns() > 0) {
+        timSort.sort(rowPage.getBuffer(), 0, rowPage.getBuffer().getActualSize(),
+            new UnsafeRowComparator(rowPage));
+      } else {
+        timSort.sort(rowPage.getBuffer(), 0, rowPage.getBuffer().getActualSize(),
+            new UnsafeRowComparatorForNormalDIms(rowPage));
+      }
+      unsafeInMemoryIntermediateFileMerger.addDataChunkToMerge(rowPage);
+    } else {
+      rowPage.freeMemory();
+    }
+    startFileBasedMerge();
+  }
+
+  private void writeData(UnsafeCarbonRowPage rowPage, File file)
+      throws CarbonSortKeyAndGroupByException {
+    DataOutputStream stream = null;
+    try {
+      // open stream
+      stream = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(file),
+          parameters.getFileWriteBufferSize()));
+      int actualSize = rowPage.getBuffer().getActualSize();
+      // 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);
+      }
+
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
+    } finally {
+      // close streams
+      CarbonUtil.closeStreams(stream);
+    }
+  }
+
+  /**
+   * This method will be used to delete sort temp location is it is exites
+   */
+  public void deleteSortLocationIfExists() {
+    CarbonDataProcessorUtil.deleteSortLocationIfExists(parameters.getTempFileLocation());
+  }
+
+  /**
+   * Below method will be used to start file based merge
+   *
+   * @throws InterruptedException
+   */
+  private void startFileBasedMerge() throws InterruptedException {
+    dataSorterAndWriterExecutorService.shutdown();
+    dataSorterAndWriterExecutorService.awaitTermination(2, TimeUnit.DAYS);
+  }
+
+  /**
+   * Observer class for thread execution
+   * In case of any failure we need stop all the running thread
+   */
+  private class ThreadStatusObserver {
+    /**
+     * Below method will be called if any thread fails during execution
+     *
+     * @param exception
+     * @throws CarbonSortKeyAndGroupByException
+     */
+    public void notifyFailed(Throwable exception) throws CarbonSortKeyAndGroupByException {
+      dataSorterAndWriterExecutorService.shutdownNow();
+      unsafeInMemoryIntermediateFileMerger.close();
+      parameters.getObserver().setFailed(true);
+      LOGGER.error(exception);
+      throw new CarbonSortKeyAndGroupByException(exception);
+    }
+  }
+
+  /**
+   * This class is responsible for sorting and writing the object
+   * array which holds the records equal to given array size
+   */
+  private class DataSorterAndWriter implements Runnable {
+    private UnsafeCarbonRowPage page;
+
+    public DataSorterAndWriter(UnsafeCarbonRowPage rowPage) {
+      this.page = rowPage;
+    }
+
+    @Override
+    public void run() {
+      try {
+        long startTime = System.currentTimeMillis();
+        TimSort<UnsafeCarbonRow, IntPointerBuffer> timSort = new TimSort<>(
+            new UnsafeIntSortDataFormat(page));
+        // if sort_columns is not none, sort by sort_columns
+        if (parameters.getNumberOfNoDictSortColumns() > 0) {
+          timSort.sort(page.getBuffer(), 0, page.getBuffer().getActualSize(),
+              new UnsafeRowComparator(page));
+        } else {
+          timSort.sort(page.getBuffer(), 0, page.getBuffer().getActualSize(),
+              new UnsafeRowComparatorForNormalDIms(page));
+        }
+        if (page.isSaveToDisk()) {
+          // create a new file every time
+          // 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);
+          writeData(page, sortTempFile);
+          LOGGER.info("Time taken to sort row page with size" + page.getBuffer().getActualSize()
+              + " and write is: " + (System.currentTimeMillis() - startTime));
+          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
+          unsafeInMemoryIntermediateFileMerger.addFileToMerge(sortTempFile);
+        } else {
+          // creating a new memory block as size is already allocated
+          // so calling lazy memory allocator
+          MemoryBlock newMemoryBlock = UnsafeSortMemoryManager.INSTANCE
+              .allocateMemoryLazy(taskId, page.getDataBlock().size());
+          // copying data from working memory manager to sortmemory manager
+          CarbonUnsafe.getUnsafe()
+              .copyMemory(page.getDataBlock().getBaseObject(), page.getDataBlock().getBaseOffset(),
+                  newMemoryBlock.getBaseObject(), newMemoryBlock.getBaseOffset(),
+                  page.getDataBlock().size());
+          // free unsafememory manager
+          page.freeMemory();
+          page.setNewDataBlock(newMemoryBlock);
+          // add sort temp filename to and arrayList. When the list size reaches 20 then
+          // intermediate merging of sort temp files will be triggered
+          page.getBuffer().loadToUnsafe();
+          unsafeInMemoryIntermediateFileMerger.addDataChunkToMerge(page);
+          LOGGER.info(
+              "Time taken to sort row page with size" + page.getBuffer().getActualSize() + "is: "
+                  + (System.currentTimeMillis() - startTime));
+        }
+      } catch (Throwable e) {
+        try {
+          threadStatusObserver.notifyFailed(e);
+        } catch (CarbonSortKeyAndGroupByException ex) {
+          LOGGER.error(e);
+        }
+      } finally {
+        semaphore.release();
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..d02be9b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/comparator/UnsafeRowComparator.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.core.util.ByteUtil.UnsafeComparer;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
+
+public class UnsafeRowComparator implements Comparator<UnsafeCarbonRow> {
+
+  /**
+   * mapping of dictionary and no dictionary of sort_columns.
+   */
+  private boolean[] noDictionarySortColumnMaping;
+
+  private Object baseObject;
+
+  public UnsafeRowComparator(UnsafeCarbonRowPage rowPage) {
+    this.noDictionarySortColumnMaping = rowPage.getNoDictionarySortColumnMapping();
+    this.baseObject = rowPage.getDataBlock().getBaseObject();
+  }
+
+  /**
+   * 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;
+  }
+
+  /**
+   * Below method will be used to compare two mdkey
+   */
+  public int compare(UnsafeCarbonRow rowL, Object baseObjectL, UnsafeCarbonRow rowR,
+      Object baseObjectR) {
+    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(baseObjectL, rowA + sizeA);
+        byte[] byteArr1 = new byte[aShort1];
+        sizeA += 2;
+        CarbonUnsafe.getUnsafe()
+            .copyMemory(baseObjectL, rowA + sizeA, byteArr1, CarbonUnsafe.BYTE_ARRAY_OFFSET,
+                aShort1);
+        sizeA += aShort1;
+
+        short aShort2 = CarbonUnsafe.getUnsafe().getShort(baseObjectR, rowB + sizeB);
+        byte[] byteArr2 = new byte[aShort2];
+        sizeB += 2;
+        CarbonUnsafe.getUnsafe()
+            .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 + sizeA);
+        sizeA += 4;
+        int dimFieldB = CarbonUnsafe.getUnsafe().getInt(baseObjectR, rowB + sizeB);
+        sizeB += 4;
+        diff = dimFieldA - dimFieldB;
+        if (diff != 0) {
+          return diff;
+        }
+      }
+    }
+
+    return diff;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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/349c59c7/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
new file mode 100644
index 0000000..686e855
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/SortTempChunkHolder.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.processing.loading.sort.unsafe.holder;
+
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+/**
+ * Interface for merging temporary sort files/ inmemory data
+ */
+public interface SortTempChunkHolder extends Comparable<SortTempChunkHolder> {
+
+  boolean hasNext();
+
+  void readRow()  throws CarbonSortKeyAndGroupByException;
+
+  Object[] getRow();
+
+  int numberOfRows();
+
+  void close();
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeCarbonRow.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeCarbonRow.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeCarbonRow.java
new file mode 100644
index 0000000..2a8dc5b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeCarbonRow.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.processing.loading.sort.unsafe.holder;
+
+public class UnsafeCarbonRow {
+
+  public long address;
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeCarbonRowForMerge.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeCarbonRowForMerge.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeCarbonRowForMerge.java
new file mode 100644
index 0000000..a4bb684
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeCarbonRowForMerge.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.holder;
+
+public class UnsafeCarbonRowForMerge extends UnsafeCarbonRow {
+
+  public byte index;
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..6b0cfa6
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeFinalMergePageHolder.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.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.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeInMemoryIntermediateDataMerger;
+import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+
+public class UnsafeFinalMergePageHolder implements SortTempChunkHolder {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeFinalMergePageHolder.class.getName());
+
+  private int counter;
+
+  private int actualSize;
+
+  private long[] mergedAddresses;
+
+  private byte[] rowPageIndexes;
+
+  private UnsafeCarbonRowPage[] rowPages;
+
+  private NewRowComparator comparator;
+
+  private Object[] currentRow;
+
+  private int columnSize;
+
+  public UnsafeFinalMergePageHolder(UnsafeInMemoryIntermediateDataMerger merger,
+      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 NewRowComparator(noDictSortColumnMapping);
+    this.columnSize = columnSize;
+  }
+
+  public boolean hasNext() {
+    if (counter < actualSize) {
+      return true;
+    }
+    return false;
+  }
+
+  public void readRow() {
+    currentRow = new Object[columnSize];
+    rowPages[rowPageIndexes[counter]].getRow(mergedAddresses[counter], currentRow);
+    counter++;
+  }
+
+  public Object[] getRow() {
+    return currentRow;
+  }
+
+  @Override public int compareTo(SortTempChunkHolder o) {
+    return comparator.compare(currentRow, o.getRow());
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof UnsafeFinalMergePageHolder)) {
+      return false;
+    }
+
+    UnsafeFinalMergePageHolder o = (UnsafeFinalMergePageHolder) obj;
+    return this == o;
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode();
+  }
+
+  public int numberOfRows() {
+    return actualSize;
+  }
+
+  public void close() {
+    for (int i = 0; i < rowPages.length; i++) {
+      rowPages[i].freeMemory();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..6f05088
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryHolder.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.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.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+
+public class UnsafeInmemoryHolder implements SortTempChunkHolder {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeInmemoryHolder.class.getName());
+
+  private int counter;
+
+  private int actualSize;
+
+  private UnsafeCarbonRowPage rowPage;
+
+  private Object[] currentRow;
+
+  private long address;
+
+  private NewRowComparator comparator;
+
+  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 NewRowComparator(rowPage.getNoDictionarySortColumnMapping());
+    this.columnSize = columnSize;
+  }
+
+  public boolean hasNext() {
+    if (counter < actualSize) {
+      return true;
+    }
+    return false;
+  }
+
+  public void readRow() {
+    currentRow = new Object[columnSize];
+    address = rowPage.getBuffer().get(counter);
+    rowPage.getRow(address + rowPage.getDataBlock().getBaseOffset(), currentRow);
+    counter++;
+  }
+
+  public Object[] getRow() {
+    return currentRow;
+  }
+
+  @Override public int compareTo(SortTempChunkHolder o) {
+    return comparator.compare(currentRow, o.getRow());
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof UnsafeInmemoryHolder)) {
+      return false;
+    }
+
+    UnsafeInmemoryHolder o = (UnsafeInmemoryHolder)obj;
+
+    return this == o;
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode();
+  }
+
+  public int numberOfRows() {
+    return actualSize;
+  }
+
+  public void close() {
+    rowPage.freeMemory();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryMergeHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryMergeHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryMergeHolder.java
new file mode 100644
index 0000000..3b9d8d7
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeInmemoryMergeHolder.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.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.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparator;
+
+/**
+ * It is used for merging unsafe inmemory intermediate data
+ */
+public class UnsafeInmemoryMergeHolder implements Comparable<UnsafeInmemoryMergeHolder> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeInmemoryMergeHolder.class.getName());
+
+  private int counter;
+
+  private int actualSize;
+
+  private UnsafeCarbonRowPage rowPage;
+
+  private UnsafeCarbonRowForMerge currentRow;
+
+  private long address;
+
+  private UnsafeRowComparator comparator;
+
+  private Object baseObject;
+
+  private byte index;
+
+  public UnsafeInmemoryMergeHolder(UnsafeCarbonRowPage rowPage, byte index) {
+    this.actualSize = rowPage.getBuffer().getActualSize();
+    this.rowPage = rowPage;
+    LOGGER.audit("Processing unsafe inmemory rows page with size : " + actualSize);
+    this.comparator = new UnsafeRowComparator(rowPage);
+    this.baseObject = rowPage.getDataBlock().getBaseObject();
+    currentRow = new UnsafeCarbonRowForMerge();
+    this.index = index;
+  }
+
+  public boolean hasNext() {
+    if (counter < actualSize) {
+      return true;
+    }
+    return false;
+  }
+
+  public void readRow() {
+    address = rowPage.getBuffer().get(counter);
+    currentRow = new UnsafeCarbonRowForMerge();
+    currentRow.address = address + rowPage.getDataBlock().getBaseOffset();
+    currentRow.index = index;
+    counter++;
+  }
+
+  public UnsafeCarbonRowForMerge getRow() {
+    return currentRow;
+  }
+
+  @Override public int compareTo(UnsafeInmemoryMergeHolder o) {
+    return comparator.compare(currentRow, baseObject, o.getRow(), o.getBaseObject());
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof UnsafeInmemoryMergeHolder)) {
+      return false;
+    }
+
+    UnsafeInmemoryMergeHolder o = (UnsafeInmemoryMergeHolder)obj;
+    return this == o;
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode();
+  }
+
+  public Object getBaseObject() {
+    return baseObject;
+  }
+
+  public void close() {
+    rowPage.freeMemory();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..331b9db
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/holder/UnsafeSortTempFileChunkHolder.java
@@ -0,0 +1,472 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort.unsafe.holder;
+
+import java.io.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+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;
+import java.util.concurrent.Future;
+
+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.metadata.datatype.DataType;
+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.sort.exception.CarbonSortKeyAndGroupByException;
+import org.apache.carbondata.processing.sort.sortdata.NewRowComparator;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+
+public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeSortTempFileChunkHolder.class.getName());
+
+  /**
+   * temp file
+   */
+  private File tempFile;
+
+  /**
+   * read stream
+   */
+  private DataInputStream stream;
+
+  /**
+   * entry count
+   */
+  private int entryCount;
+
+  /**
+   * return row
+   */
+  private Object[] returnRow;
+
+  /**
+   * number of measures
+   */
+  private int measureCount;
+
+  /**
+   * number of dimensionCount
+   */
+  private int dimensionCount;
+
+  /**
+   * number of complexDimensionCount
+   */
+  private int complexDimensionCount;
+
+  /**
+   * fileBufferSize for file reader stream size
+   */
+  private int fileBufferSize;
+
+  private Object[][] currentBuffer;
+
+  private Object[][] backupBuffer;
+
+  private boolean isBackupFilled;
+
+  private boolean prefetch;
+
+  private int bufferSize;
+
+  private int bufferRowCounter;
+
+  private ExecutorService executorService;
+
+  private Future<Void> submit;
+
+  private int prefetchRecordsProceesed;
+
+  /**
+   * sortTempFileNoOFRecordsInCompression
+   */
+  private int sortTempFileNoOFRecordsInCompression;
+
+  /**
+   * isSortTempFileCompressionEnabled
+   */
+  private boolean isSortTempFileCompressionEnabled;
+
+  /**
+   * totalRecordFetch
+   */
+  private int totalRecordFetch;
+
+  private int noDictionaryCount;
+
+  private DataType[] measureDataType;
+
+  private int numberOfObjectRead;
+  /**
+   * to store whether dimension is of dictionary type or not
+   */
+  private boolean[] isNoDictionaryDimensionColumn;
+
+  private int nullSetWordsLength;
+
+  private Comparator<Object[]> comparator;
+
+  /**
+   * Constructor to initialize
+   */
+  public UnsafeSortTempFileChunkHolder(File tempFile, SortParameters parameters) {
+    // set temp file
+    this.tempFile = tempFile;
+
+    // set measure and dimension count
+    this.measureCount = parameters.getMeasureColCount();
+    this.dimensionCount = parameters.getDimColCount();
+    this.complexDimensionCount = parameters.getComplexDimColCount();
+
+    this.noDictionaryCount = parameters.getNoDictionaryCount();
+    // set mdkey length
+    this.fileBufferSize = parameters.getFileBufferSize();
+    this.executorService = Executors.newFixedThreadPool(1);
+    this.measureDataType = parameters.getMeasureDataType();
+    this.isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
+    this.nullSetWordsLength = ((measureCount - 1) >> 6) + 1;
+    comparator = new NewRowComparator(parameters.getNoDictionarySortColumn());
+    initialize();
+  }
+
+  /**
+   * This method will be used to initialize
+   *
+   * @throws CarbonSortKeyAndGroupByException problem while initializing
+   */
+  public void initialize() {
+    prefetch = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_PREFETCH,
+            CarbonCommonConstants.CARBON_MERGE_SORT_PREFETCH_DEFAULT));
+    bufferSize = Integer.parseInt(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
+            CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT));
+    this.isSortTempFileCompressionEnabled = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED,
+            CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED_DEFAULTVALUE));
+    if (this.isSortTempFileCompressionEnabled) {
+      LOGGER.info("Compression was used while writing the sortTempFile");
+    }
+
+    try {
+      this.sortTempFileNoOFRecordsInCompression = Integer.parseInt(CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION,
+              CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE));
+      if (this.sortTempFileNoOFRecordsInCompression < 1) {
+        LOGGER.error("Invalid value for: "
+            + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
+            + ": Only Positive Integer value(greater than zero) is allowed.Default value will"
+            + " be used");
+
+        this.sortTempFileNoOFRecordsInCompression = Integer.parseInt(
+            CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
+      }
+    } catch (NumberFormatException e) {
+      LOGGER.error(
+          "Invalid value for: " + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
+              + ", only Positive Integer value is allowed.Default value will be used");
+      this.sortTempFileNoOFRecordsInCompression = Integer
+          .parseInt(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
+    }
+
+    initialise();
+  }
+
+  private void initialise() {
+    try {
+      if (isSortTempFileCompressionEnabled) {
+        this.bufferSize = sortTempFileNoOFRecordsInCompression;
+      }
+      stream = new DataInputStream(
+          new BufferedInputStream(new FileInputStream(tempFile), this.fileBufferSize));
+      this.entryCount = stream.readInt();
+      LOGGER.audit("Processing unsafe mode file rows with size : " + entryCount);
+      if (prefetch) {
+        new DataFetcher(false).call();
+        totalRecordFetch += currentBuffer.length;
+        if (totalRecordFetch < this.entryCount) {
+          submit = executorService.submit(new DataFetcher(true));
+        }
+      } else {
+        if (isSortTempFileCompressionEnabled) {
+          new DataFetcher(false).call();
+        }
+      }
+
+    } catch (FileNotFoundException e) {
+      LOGGER.error(e);
+      throw new RuntimeException(tempFile + " No Found", e);
+    } catch (IOException e) {
+      LOGGER.error(e);
+      throw new RuntimeException(tempFile + " No Found", e);
+    } catch (Exception e) {
+      LOGGER.error(e);
+      throw new RuntimeException(tempFile + " Problem while reading", e);
+    }
+  }
+
+  /**
+   * This method will be used to read new row from file
+   *
+   * @throws CarbonSortKeyAndGroupByException problem while reading
+   */
+  public void readRow() throws CarbonSortKeyAndGroupByException {
+    if (prefetch) {
+      fillDataForPrefetch();
+    } else if (isSortTempFileCompressionEnabled) {
+      if (bufferRowCounter >= bufferSize) {
+        try {
+          new DataFetcher(false).call();
+          bufferRowCounter = 0;
+        } catch (Exception e) {
+          LOGGER.error(e);
+          throw new CarbonSortKeyAndGroupByException(tempFile + " Problem while reading", e);
+        }
+
+      }
+      prefetchRecordsProceesed++;
+      returnRow = currentBuffer[bufferRowCounter++];
+    } else {
+      this.returnRow = getRowFromStream();
+    }
+  }
+
+  private void fillDataForPrefetch() {
+    if (bufferRowCounter >= bufferSize) {
+      if (isBackupFilled) {
+        bufferRowCounter = 0;
+        currentBuffer = backupBuffer;
+        totalRecordFetch += currentBuffer.length;
+        isBackupFilled = false;
+        if (totalRecordFetch < this.entryCount) {
+          submit = executorService.submit(new DataFetcher(true));
+        }
+      } else {
+        try {
+          submit.get();
+        } catch (Exception e) {
+          LOGGER.error(e);
+        }
+        bufferRowCounter = 0;
+        currentBuffer = backupBuffer;
+        isBackupFilled = false;
+        totalRecordFetch += currentBuffer.length;
+        if (totalRecordFetch < this.entryCount) {
+          submit = executorService.submit(new DataFetcher(true));
+        }
+      }
+    }
+    prefetchRecordsProceesed++;
+    returnRow = currentBuffer[bufferRowCounter++];
+  }
+
+  /**
+   * @return
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
+    Object[] row = new Object[dimensionCount + measureCount];
+    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 < dimensionCount; 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 < measureCount; mesCount++) {
+        if (UnsafeCarbonRowPage.isSet(words, mesCount)) {
+          switch (measureDataType[mesCount]) {
+            case SHORT:
+              row[dimensionCount + mesCount] = stream.readShort();
+              break;
+            case INT:
+              row[dimensionCount + mesCount] = stream.readInt();
+              break;
+            case LONG:
+              row[dimensionCount + mesCount] = stream.readLong();
+              break;
+            case DOUBLE:
+              row[dimensionCount + mesCount] = stream.readDouble();
+              break;
+            case DECIMAL:
+              short aShort = stream.readShort();
+              byte[] bigDecimalInBytes = new byte[aShort];
+              stream.readFully(bigDecimalInBytes);
+              row[dimensionCount + mesCount] = DataTypeUtil.byteToBigDecimal(bigDecimalInBytes);
+              break;
+            default:
+              throw new IllegalArgumentException("unsupported data type:" +
+                  measureDataType[mesCount]);
+          }
+        }
+      }
+      return row;
+    } catch (IOException e) {
+      throw new CarbonSortKeyAndGroupByException(e);
+    }
+  }
+
+  /**
+   * below method will be used to get the row
+   *
+   * @return row
+   */
+  public Object[] getRow() {
+    return this.returnRow;
+  }
+
+  /**
+   * below method will be used to check whether any more records are present
+   * in file or not
+   *
+   * @return more row present in file
+   */
+  public boolean hasNext() {
+    if (prefetch || isSortTempFileCompressionEnabled) {
+      return this.prefetchRecordsProceesed < this.entryCount;
+    }
+    return this.numberOfObjectRead < this.entryCount;
+  }
+
+  /**
+   * Below method will be used to close streams
+   */
+  public void close() {
+    CarbonUtil.closeStreams(stream);
+    executorService.shutdown();
+  }
+
+  /**
+   * This method will number of entries
+   *
+   * @return entryCount
+   */
+  public int numberOfRows() {
+    return entryCount;
+  }
+
+  @Override public int compareTo(SortTempChunkHolder other) {
+    return comparator.compare(returnRow, other.getRow());
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof UnsafeSortTempFileChunkHolder)) {
+      return false;
+    }
+    UnsafeSortTempFileChunkHolder o = (UnsafeSortTempFileChunkHolder) obj;
+
+    return this == o;
+  }
+
+  @Override public int hashCode() {
+    int hash = 0;
+    hash += 31 * measureCount;
+    hash += 31 * dimensionCount;
+    hash += 31 * complexDimensionCount;
+    hash += 31 * noDictionaryCount;
+    hash += tempFile.hashCode();
+    return hash;
+  }
+
+  private final class DataFetcher implements Callable<Void> {
+    private boolean isBackUpFilling;
+
+    private int numberOfRecords;
+
+    private DataFetcher(boolean backUp) {
+      isBackUpFilling = backUp;
+      calculateNumberOfRecordsToBeFetched();
+    }
+
+    private void calculateNumberOfRecordsToBeFetched() {
+      int numberOfRecordsLeftToBeRead = entryCount - totalRecordFetch;
+      numberOfRecords =
+          bufferSize < numberOfRecordsLeftToBeRead ? bufferSize : numberOfRecordsLeftToBeRead;
+    }
+
+    @Override public Void call() throws Exception {
+      try {
+        if (isBackUpFilling) {
+          backupBuffer = prefetchRecordsFromFile(numberOfRecords);
+          isBackupFilled = true;
+        } else {
+          currentBuffer = prefetchRecordsFromFile(numberOfRecords);
+        }
+      } catch (Exception e) {
+        LOGGER.error(e);
+      }
+      return null;
+    }
+
+  }
+
+  /**
+   * This method will read the records from sort temp file and keep it in a buffer
+   *
+   * @param numberOfRecords
+   * @return
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeInMemoryIntermediateDataMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeInMemoryIntermediateDataMerger.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeInMemoryIntermediateDataMerger.java
new file mode 100644
index 0000000..3955864
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeInMemoryIntermediateDataMerger.java
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.merger;
+
+import java.util.AbstractQueue;
+import java.util.PriorityQueue;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRowForMerge;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeInmemoryMergeHolder;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+public class UnsafeInMemoryIntermediateDataMerger implements Runnable {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeInMemoryIntermediateDataMerger.class.getName());
+
+  /**
+   * recordHolderHeap
+   */
+  private AbstractQueue<UnsafeInmemoryMergeHolder> recordHolderHeap;
+
+  /**
+   * fileCounter
+   */
+  private int holderCounter;
+
+  /**
+   * entryCount
+   */
+  private int entryCount;
+
+  private UnsafeCarbonRowPage[] unsafeCarbonRowPages;
+
+  private long[] mergedAddresses;
+
+  private byte[] rowPageIndexes;
+
+  /**
+   * IntermediateFileMerger Constructor
+   */
+  public UnsafeInMemoryIntermediateDataMerger(UnsafeCarbonRowPage[] unsafeCarbonRowPages,
+      int totalSize) {
+    this.holderCounter = unsafeCarbonRowPages.length;
+    this.unsafeCarbonRowPages = unsafeCarbonRowPages;
+    this.mergedAddresses = new long[totalSize];
+    this.rowPageIndexes = new byte[totalSize];
+    this.entryCount = 0;
+  }
+
+  @Override
+  public void run() {
+    long intermediateMergeStartTime = System.currentTimeMillis();
+    int holderCounterConst = holderCounter;
+    try {
+      startSorting();
+      while (hasNext()) {
+        writeDataToMemory(next());
+      }
+      double intermediateMergeCostTime =
+          (System.currentTimeMillis() - intermediateMergeStartTime) / 1000.0;
+      LOGGER.info("============================== Intermediate Merge of " + holderCounterConst
+          + " in-memory sort Cost Time: " + intermediateMergeCostTime + "(s)");
+    } catch (Exception e) {
+      LOGGER.error(e, "Problem while intermediate merging");
+    }
+  }
+
+  /**
+   * This method will be used to get the sorted record from file
+   *
+   * @return sorted record sorted record
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private UnsafeCarbonRowForMerge getSortedRecordFromMemory()
+      throws CarbonSortKeyAndGroupByException {
+    UnsafeCarbonRowForMerge row = null;
+
+    // poll the top object from heap
+    // heap maintains binary tree which is based on heap condition that will
+    // be based on comparator we are passing the heap
+    // when will call poll it will always delete root of the tree and then
+    // it does trickel down operation complexity is log(n)
+    UnsafeInmemoryMergeHolder poll = this.recordHolderHeap.poll();
+
+    // get the row from chunk
+    row = poll.getRow();
+
+    // check if there no entry present
+    if (!poll.hasNext()) {
+      // change the file counter
+      --this.holderCounter;
+
+      // reaturn row
+      return row;
+    }
+
+    // read new row
+    poll.readRow();
+
+    // add to heap
+    this.recordHolderHeap.add(poll);
+
+    // return row
+    return row;
+  }
+
+  /**
+   * Below method will be used to start storing process This method will get
+   * all the temp files present in sort temp folder then it will create the
+   * record holder heap and then it will read first record from each file and
+   * initialize the heap
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private void startSorting() throws CarbonSortKeyAndGroupByException {
+    LOGGER.info("Number of row pages in intermediate merger: " + this.holderCounter);
+
+    // create record holder heap
+    createRecordHolderQueue(unsafeCarbonRowPages);
+
+    // iterate over file list and create chunk holder and add to heap
+    LOGGER.info("Started adding first record from row page");
+
+    UnsafeInmemoryMergeHolder unsafePageHolder = null;
+    byte index = 0;
+    for (UnsafeCarbonRowPage unsafeCarbonRowPage : unsafeCarbonRowPages) {
+      // create chunk holder
+      unsafePageHolder = new UnsafeInmemoryMergeHolder(unsafeCarbonRowPage, index++);
+
+      // initialize
+      unsafePageHolder.readRow();
+
+      // add to heap
+      this.recordHolderHeap.add(unsafePageHolder);
+    }
+
+    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
+  }
+
+  /**
+   * This method will be used to create the heap which will be used to hold
+   * the chunk of data
+   */
+  private void createRecordHolderQueue(UnsafeCarbonRowPage[] pages) {
+    // creating record holder heap
+    this.recordHolderHeap = new PriorityQueue<UnsafeInmemoryMergeHolder>(pages.length);
+  }
+
+  /**
+   * This method will be used to get the sorted row
+   *
+   * @return sorted row
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private UnsafeCarbonRowForMerge next() throws CarbonSortKeyAndGroupByException {
+    return getSortedRecordFromMemory();
+  }
+
+  /**
+   * This method will be used to check whether any more element is present or
+   * not
+   *
+   * @return more element is present
+   */
+  private boolean hasNext() {
+    return this.holderCounter > 0;
+  }
+
+  /**
+   * Below method will be used to write data to file
+   */
+  private void writeDataToMemory(UnsafeCarbonRowForMerge row) {
+    mergedAddresses[entryCount] = row.address;
+    rowPageIndexes[entryCount] = row.index;
+    entryCount++;
+  }
+
+  public int getEntryCount() {
+    return entryCount;
+  }
+
+  public UnsafeCarbonRowPage[] getUnsafeCarbonRowPages() {
+    return unsafeCarbonRowPages;
+  }
+
+  public long[] getMergedAddresses() {
+    return mergedAddresses;
+  }
+
+  public byte[] getRowPageIndexes() {
+    return rowPageIndexes;
+  }
+}


[26/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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/349c59c7/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
new file mode 100644
index 0000000..6d6ff94
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SingleThreadFinalSortFilesMerger.java
@@ -0,0 +1,312 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.File;
+import java.io.FileFilter;
+import java.util.AbstractQueue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.PriorityQueue;
+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.exception.CarbonDataWriterException;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(SingleThreadFinalSortFilesMerger.class.getName());
+
+  /**
+   * lockObject
+   */
+  private static final Object LOCKOBJECT = new Object();
+
+  /**
+   * fileCounter
+   */
+  private int fileCounter;
+
+  /**
+   * fileBufferSize
+   */
+  private int fileBufferSize;
+
+  /**
+   * recordHolderHeap
+   */
+  private AbstractQueue<SortTempFileChunkHolder> recordHolderHeapLocal;
+
+  /**
+   * tableName
+   */
+  private String tableName;
+
+  /**
+   * measureCount
+   */
+  private int measureCount;
+
+  /**
+   * dimensionCount
+   */
+  private int dimensionCount;
+
+  /**
+   * measure count
+   */
+  private int noDictionaryCount;
+
+  /**
+   * complexDimensionCount
+   */
+  private int complexDimensionCount;
+
+  /**
+   * tempFileLocation
+   */
+  private String[] tempFileLocation;
+
+  private DataType[] measureDataType;
+
+  /**
+   * below code is to check whether dimension
+   * is of no dictionary type or not
+   */
+  private boolean[] isNoDictionaryColumn;
+
+  private boolean[] isNoDictionarySortColumn;
+
+  public SingleThreadFinalSortFilesMerger(String[] tempFileLocation, String tableName,
+      int dimensionCount, int complexDimensionCount, int measureCount, int noDictionaryCount,
+      DataType[] type, boolean[] isNoDictionaryColumn, boolean[] isNoDictionarySortColumn) {
+    this.tempFileLocation = tempFileLocation;
+    this.tableName = tableName;
+    this.dimensionCount = dimensionCount;
+    this.complexDimensionCount = complexDimensionCount;
+    this.measureCount = measureCount;
+    this.measureDataType = type;
+    this.noDictionaryCount = noDictionaryCount;
+    this.isNoDictionaryColumn = isNoDictionaryColumn;
+    this.isNoDictionarySortColumn = isNoDictionarySortColumn;
+  }
+
+  /**
+   * This method will be used to merger the merged files
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  public void startFinalMerge() throws CarbonDataWriterException {
+    List<File> filesToMerge = getFilesToMergeSort();
+    if (filesToMerge.size() == 0)
+    {
+      LOGGER.info("No file to merge in final merge stage");
+      return;
+    }
+
+    startSorting(filesToMerge);
+  }
+
+  private List<File> getFilesToMergeSort() {
+    FileFilter fileFilter = new FileFilter() {
+      public boolean accept(File pathname) {
+        return pathname.getName().startsWith(tableName);
+      }
+    };
+
+    // get all the merged files
+    List<File> files = new ArrayList<File>(tempFileLocation.length);
+    for (String tempLoc : tempFileLocation)
+    {
+      File[] subFiles = new File(tempLoc).listFiles(fileFilter);
+      if (null != subFiles && subFiles.length > 0)
+      {
+        files.addAll(Arrays.asList(subFiles));
+      }
+    }
+
+    return files;
+  }
+
+  /**
+   * Below method will be used to start storing process This method will get
+   * all the temp files present in sort temp folder then it will create the
+   * record holder heap and then it will read first record from each file and
+   * initialize the heap
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private void startSorting(List<File> files) throws CarbonDataWriterException {
+    this.fileCounter = files.size();
+    if (fileCounter == 0) {
+      LOGGER.info("No files to merge sort");
+      return;
+    }
+    this.fileBufferSize = CarbonDataProcessorUtil
+        .getFileBufferSize(this.fileCounter, CarbonProperties.getInstance(),
+            CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+    LOGGER.info("Number of temp file: " + this.fileCounter);
+
+    LOGGER.info("File Buffer Size: " + this.fileBufferSize);
+
+    // create record holder heap
+    createRecordHolderQueue();
+
+    // iterate over file list and create chunk holder and add to heap
+    LOGGER.info("Started adding first record from each file");
+    int maxThreadForSorting = 0;
+    try {
+      maxThreadForSorting = Integer.parseInt(CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD,
+              CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE));
+    } catch (NumberFormatException e) {
+      maxThreadForSorting =
+          Integer.parseInt(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE);
+    }
+    ExecutorService service = Executors.newFixedThreadPool(maxThreadForSorting);
+
+    for (final File tempFile : files) {
+
+      Runnable runnable = new Runnable() {
+        @Override public void run() {
+
+            // create chunk holder
+            SortTempFileChunkHolder sortTempFileChunkHolder =
+                new SortTempFileChunkHolder(tempFile, dimensionCount, complexDimensionCount,
+                    measureCount, fileBufferSize, noDictionaryCount, measureDataType,
+                    isNoDictionaryColumn, isNoDictionarySortColumn);
+          try {
+            // initialize
+            sortTempFileChunkHolder.initialize();
+            sortTempFileChunkHolder.readRow();
+          } catch (CarbonSortKeyAndGroupByException ex) {
+            LOGGER.error(ex);
+          }
+
+          synchronized (LOCKOBJECT) {
+            recordHolderHeapLocal.add(sortTempFileChunkHolder);
+          }
+        }
+      };
+      service.execute(runnable);
+    }
+    service.shutdown();
+
+    try {
+      service.awaitTermination(2, TimeUnit.HOURS);
+    } catch (Exception e) {
+      throw new CarbonDataWriterException(e.getMessage(), e);
+    }
+
+    LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
+  }
+
+  /**
+   * This method will be used to create the heap which will be used to hold
+   * the chunk of data
+   */
+  private void createRecordHolderQueue() {
+    // creating record holder heap
+    this.recordHolderHeapLocal = new PriorityQueue<SortTempFileChunkHolder>(fileCounter);
+  }
+
+  /**
+   * This method will be used to get the sorted row
+   *
+   * @return sorted row
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  public Object[] next() {
+    return getSortedRecordFromFile();
+  }
+
+  /**
+   * This method will be used to get the sorted record from file
+   *
+   * @return sorted record sorted record
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  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
+    // be based on comparator we are passing the heap
+    // when will call poll it will always delete root of the tree and then
+    // it does trickel down operation complexity is log(n)
+    SortTempFileChunkHolder poll = this.recordHolderHeapLocal.poll();
+
+    // get the row from chunk
+    row = poll.getRow();
+
+    // check if there no entry present
+    if (!poll.hasNext()) {
+      // if chunk is empty then close the stream
+      poll.closeStream();
+
+      // change the file counter
+      --this.fileCounter;
+
+      // reaturn row
+      return row;
+    }
+
+    // read new row
+    try {
+      poll.readRow();
+    } catch (CarbonSortKeyAndGroupByException e) {
+      throw new CarbonDataWriterException(e.getMessage(), e);
+    }
+
+    // add to heap
+    this.recordHolderHeapLocal.add(poll);
+
+    // return row
+    return row;
+  }
+
+  /**
+   * This method will be used to check whether any more element is present or
+   * not
+   *
+   * @return more element is present
+   */
+  public boolean hasNext() {
+    return this.fileCounter > 0;
+  }
+
+  public void clear() {
+    if (null != recordHolderHeapLocal) {
+      recordHolderHeapLocal = null;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..fc744a6
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortDataRows.java
@@ -0,0 +1,437 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+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.metadata.datatype.DataType;
+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.sort.exception.CarbonSortKeyAndGroupByException;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+public class SortDataRows {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(SortDataRows.class.getName());
+  /**
+   * entryCount
+   */
+  private int entryCount;
+  /**
+   * record holder array
+   */
+  private Object[][] recordHolderList;
+  /**
+   * threadStatusObserver
+   */
+  private ThreadStatusObserver threadStatusObserver;
+  /**
+   * executor service for data sort holder
+   */
+  private ExecutorService dataSorterAndWriterExecutorService;
+  /**
+   * semaphore which will used for managing sorted data object arrays
+   */
+  private Semaphore semaphore;
+
+  private SortParameters parameters;
+
+  private int sortBufferSize;
+
+  private SortIntermediateFileMerger intermediateFileMerger;
+
+  private final Object addRowsLock = new Object();
+
+  public SortDataRows(SortParameters parameters,
+      SortIntermediateFileMerger intermediateFileMerger) {
+    this.parameters = parameters;
+
+    this.intermediateFileMerger = intermediateFileMerger;
+
+    int batchSize = CarbonProperties.getInstance().getBatchSize();
+
+    this.sortBufferSize = Math.max(parameters.getSortBufferSize(), batchSize);
+    // observer of writing file in thread
+    this.threadStatusObserver = new ThreadStatusObserver();
+  }
+
+  /**
+   * This method will be used to initialize
+   */
+  public void initialize() throws CarbonSortKeyAndGroupByException {
+
+    // create holder list which will hold incoming rows
+    // size of list will be sort buffer size + 1 to avoid creation of new
+    // array in list array
+    this.recordHolderList = new Object[sortBufferSize][];
+    // Delete if any older file exists in sort temp folder
+    deleteSortLocationIfExists();
+
+    // create new sort temp directory
+    CarbonDataProcessorUtil.createLocations(parameters.getTempFileLocation());
+    this.dataSorterAndWriterExecutorService =
+        Executors.newFixedThreadPool(parameters.getNumberOfCores());
+    semaphore = new Semaphore(parameters.getNumberOfCores());
+  }
+
+  /**
+   * This method will be used to add new row
+   *
+   * @param row new row
+   * @throws CarbonSortKeyAndGroupByException problem while writing
+   */
+  public void addRow(Object[] row) throws CarbonSortKeyAndGroupByException {
+    // 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
+    int currentSize = entryCount;
+
+    if (sortBufferSize == currentSize) {
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("************ Writing to temp file ********** ");
+      }
+      intermediateFileMerger.startMergingIfPossible();
+      Object[][] recordHolderListLocal = recordHolderList;
+      try {
+        semaphore.acquire();
+        dataSorterAndWriterExecutorService.execute(new DataSorterAndWriter(recordHolderListLocal));
+      } catch (InterruptedException e) {
+        LOGGER.error(
+            "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
+        throw new CarbonSortKeyAndGroupByException(e.getMessage());
+      }
+      // create the new holder Array
+      this.recordHolderList = new Object[this.sortBufferSize][];
+      this.entryCount = 0;
+    }
+    recordHolderList[entryCount++] = row;
+  }
+
+  /**
+   * This method will be used to add new row
+   *
+   * @param rowBatch new rowBatch
+   * @throws CarbonSortKeyAndGroupByException problem while writing
+   */
+  public void addRowBatch(Object[][] rowBatch, int size) throws CarbonSortKeyAndGroupByException {
+    // 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
+    synchronized (addRowsLock) {
+      int sizeLeft = 0;
+      if (entryCount + size >= sortBufferSize) {
+        if (LOGGER.isDebugEnabled()) {
+          LOGGER.debug("************ Writing to temp file ********** ");
+        }
+        intermediateFileMerger.startMergingIfPossible();
+        Object[][] recordHolderListLocal = recordHolderList;
+        sizeLeft = sortBufferSize - entryCount ;
+        if (sizeLeft > 0) {
+          System.arraycopy(rowBatch, 0, recordHolderListLocal, entryCount, sizeLeft);
+        }
+        try {
+          semaphore.acquire();
+          dataSorterAndWriterExecutorService
+              .execute(new DataSorterAndWriter(recordHolderListLocal));
+        } catch (Exception e) {
+          LOGGER.error(
+              "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
+          throw new CarbonSortKeyAndGroupByException(e);
+        }
+        // create the new holder Array
+        this.recordHolderList = new Object[this.sortBufferSize][];
+        this.entryCount = 0;
+        size = size - sizeLeft;
+        if (size == 0) {
+          return;
+        }
+      }
+      System.arraycopy(rowBatch, sizeLeft, recordHolderList, entryCount, size);
+      entryCount += size;
+    }
+  }
+
+  /**
+   * Below method will be used to start storing process This method will get
+   * all the temp files present in sort temp folder then it will create the
+   * record holder heap and then it will read first record from each file and
+   * initialize the heap
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  public void startSorting() throws CarbonSortKeyAndGroupByException {
+    LOGGER.info("File based sorting will be used");
+    if (this.entryCount > 0) {
+      Object[][] toSort;
+      toSort = new Object[entryCount][];
+      System.arraycopy(recordHolderList, 0, toSort, 0, entryCount);
+      if (parameters.getNumberOfNoDictSortColumns() > 0) {
+        Arrays.sort(toSort, new NewRowComparator(parameters.getNoDictionarySortColumn()));
+      } else {
+        Arrays.sort(toSort, new NewRowComparatorForNormalDims(parameters.getNumberOfSortColumns()));
+      }
+      recordHolderList = toSort;
+
+      // 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);
+      writeDataTofile(recordHolderList, this.entryCount, file);
+
+    }
+
+    startFileBasedMerge();
+    this.recordHolderList = null;
+  }
+
+  /**
+   * Below method will be used to write data to file
+   *
+   * @throws CarbonSortKeyAndGroupByException problem while writing
+   */
+  private void writeDataTofile(Object[][] recordHolderList, int entryCountLocal, File file)
+      throws CarbonSortKeyAndGroupByException {
+    // stream
+    if (parameters.isSortFileCompressionEnabled() || parameters.isPrefetch()) {
+      writeSortTempFile(recordHolderList, entryCountLocal, file);
+      return;
+    }
+    writeData(recordHolderList, entryCountLocal, file);
+  }
+
+  private void writeSortTempFile(Object[][] recordHolderList, int entryCountLocal, File file)
+      throws CarbonSortKeyAndGroupByException {
+    TempSortFileWriter writer = null;
+
+    try {
+      writer = getWriter();
+      writer.initiaize(file, entryCountLocal);
+      writer.writeSortTempFile(recordHolderList);
+    } catch (CarbonSortKeyAndGroupByException e) {
+      LOGGER.error(e, "Problem while writing the sort temp file");
+      throw e;
+    } finally {
+      if (writer != null) {
+        writer.finish();
+      }
+    }
+  }
+
+  private void writeData(Object[][] recordHolderList, int entryCountLocal, File file)
+      throws CarbonSortKeyAndGroupByException {
+    DataOutputStream stream = null;
+    try {
+      // open stream
+      stream = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(file),
+          parameters.getFileWriteBufferSize()));
+
+      // 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);
+            switch (type[mesCount]) {
+              case SHORT:
+                stream.writeShort((Short) value);
+                break;
+              case INT:
+                stream.writeInt((Integer) value);
+                break;
+              case LONG:
+                stream.writeLong((Long) value);
+                break;
+              case DOUBLE:
+                stream.writeDouble((Double) value);
+                break;
+              case DECIMAL:
+                BigDecimal val = (BigDecimal) value;
+                byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
+                stream.writeInt(bigDecimalInBytes.length);
+                stream.write(bigDecimalInBytes);
+                break;
+              default:
+                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);
+    } finally {
+      // close streams
+      CarbonUtil.closeStreams(stream);
+    }
+  }
+
+  private TempSortFileWriter getWriter() {
+    TempSortFileWriter chunkWriter = null;
+    TempSortFileWriter writer = TempSortFileWriterFactory.getInstance()
+        .getTempSortFileWriter(parameters.isSortFileCompressionEnabled(),
+            parameters.getDimColCount(), parameters.getComplexDimColCount(),
+            parameters.getMeasureColCount(), parameters.getNoDictionaryCount(),
+            parameters.getFileWriteBufferSize());
+
+    if (parameters.isPrefetch() && !parameters.isSortFileCompressionEnabled()) {
+      chunkWriter = new SortTempFileChunkWriter(writer, parameters.getBufferSize());
+    } else {
+      chunkWriter =
+          new SortTempFileChunkWriter(writer, parameters.getSortTempFileNoOFRecordsInCompression());
+    }
+
+    return chunkWriter;
+  }
+
+  /**
+   * This method will be used to delete sort temp location is it is exites
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  public void deleteSortLocationIfExists() throws CarbonSortKeyAndGroupByException {
+    CarbonDataProcessorUtil.deleteSortLocationIfExists(parameters.getTempFileLocation());
+  }
+
+  /**
+   * Below method will be used to start file based merge
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private void startFileBasedMerge() throws CarbonSortKeyAndGroupByException {
+    try {
+      dataSorterAndWriterExecutorService.shutdown();
+      dataSorterAndWriterExecutorService.awaitTermination(2, TimeUnit.DAYS);
+    } catch (InterruptedException e) {
+      throw new CarbonSortKeyAndGroupByException("Problem while shutdown the server ", e);
+    }
+  }
+
+  /**
+   * Observer class for thread execution
+   * In case of any failure we need stop all the running thread
+   */
+  private class ThreadStatusObserver {
+    /**
+     * Below method will be called if any thread fails during execution
+     *
+     * @param exception
+     * @throws CarbonSortKeyAndGroupByException
+     */
+    public void notifyFailed(Throwable exception) throws CarbonSortKeyAndGroupByException {
+      dataSorterAndWriterExecutorService.shutdownNow();
+      intermediateFileMerger.close();
+      parameters.getObserver().setFailed(true);
+      LOGGER.error(exception);
+      throw new CarbonSortKeyAndGroupByException(exception);
+    }
+  }
+
+  /**
+   * This class is responsible for sorting and writing the object
+   * array which holds the records equal to given array size
+   */
+  private class DataSorterAndWriter implements Runnable {
+    private Object[][] recordHolderArray;
+
+    public DataSorterAndWriter(Object[][] recordHolderArray) {
+      this.recordHolderArray = recordHolderArray;
+    }
+
+    @Override
+    public void run() {
+      try {
+        long startTime = System.currentTimeMillis();
+        if (parameters.getNumberOfNoDictSortColumns() > 0) {
+          Arrays.sort(recordHolderArray,
+              new NewRowComparator(parameters.getNoDictionarySortColumn()));
+        } else {
+          Arrays.sort(recordHolderArray,
+              new NewRowComparatorForNormalDims(parameters.getNumberOfSortColumns()));
+        }
+
+        // create a new file and choose folder randomly every time
+        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);
+        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
+        intermediateFileMerger.addFileToMerge(sortTempFile);
+        LOGGER.info("Time taken to sort and write sort temp file " + sortTempFile + " is: " + (
+            System.currentTimeMillis() - startTime));
+      } catch (Throwable e) {
+        try {
+          threadStatusObserver.notifyFailed(e);
+        } catch (CarbonSortKeyAndGroupByException ex) {
+          LOGGER.error(ex);
+        }
+      } finally {
+        semaphore.release();
+      }
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..d234ce2
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortIntermediateFileMerger.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.processing.sort.sortdata;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+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.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+/**
+ * It does mergesort intermediate files to big file.
+ */
+public class SortIntermediateFileMerger {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(SortIntermediateFileMerger.class.getName());
+
+  /**
+   * executorService
+   */
+  private ExecutorService executorService;
+  /**
+   * procFiles
+   */
+  private List<File> procFiles;
+
+  private SortParameters parameters;
+
+  private final Object lockObject = new Object();
+
+  public SortIntermediateFileMerger(SortParameters parameters) {
+    this.parameters = parameters;
+    // processed file list
+    this.procFiles = new ArrayList<File>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    this.executorService = Executors.newFixedThreadPool(parameters.getNumberOfCores());
+  }
+
+  public void addFileToMerge(File sortTempFile) {
+    // add sort temp filename to and arrayList. When the list size reaches 20 then
+    // intermediate merging of sort temp files will be triggered
+    synchronized (lockObject) {
+      procFiles.add(sortTempFile);
+    }
+  }
+
+  public void startMergingIfPossible() {
+    File[] fileList;
+    if (procFiles.size() >= parameters.getNumberOfIntermediateFileToBeMerged()) {
+      synchronized (lockObject) {
+        fileList = procFiles.toArray(new File[procFiles.size()]);
+        this.procFiles = new ArrayList<File>();
+      }
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("Sumitting request for intermediate merging no of files: " + fileList.length);
+      }
+      startIntermediateMerging(fileList);
+    }
+  }
+
+  /**
+   * Below method will be used to start the intermediate file merging
+   *
+   * @param intermediateFiles
+   */
+  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);
+    IntermediateFileMerger merger = new IntermediateFileMerger(parameters, intermediateFiles, file);
+    executorService.execute(merger);
+  }
+
+  public void finish() throws CarbonSortKeyAndGroupByException {
+    try {
+      executorService.shutdown();
+      executorService.awaitTermination(2, TimeUnit.DAYS);
+    } catch (InterruptedException e) {
+      throw new CarbonSortKeyAndGroupByException("Problem while shutdown the server ", e);
+    }
+    procFiles.clear();
+    procFiles = null;
+  }
+
+  public void close() {
+    if (executorService.isShutdown()) {
+      executorService.shutdownNow();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortObserver.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortObserver.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortObserver.java
new file mode 100644
index 0000000..681e60b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortObserver.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.sort.sortdata;
+
+import java.io.Serializable;
+
+public class SortObserver implements Serializable {
+  /**
+   * is failed
+   */
+  private boolean isFailed;
+
+  /**
+   * @return the isFailed
+   */
+  public boolean isFailed() {
+    return isFailed;
+  }
+
+  /**
+   * @param isFailed the isFailed to set
+   */
+  public void setFailed(boolean isFailed) {
+    this.isFailed = isFailed;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..39e1049
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortParameters.java
@@ -0,0 +1,602 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.File;
+import java.io.Serializable;
+
+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.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.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import org.apache.commons.lang3.StringUtils;
+
+public class SortParameters implements Serializable {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(SortParameters.class.getName());
+  /**
+   * tempFileLocation
+   */
+  private String[] tempFileLocation;
+  /**
+   * sortBufferSize
+   */
+  private int sortBufferSize;
+  /**
+   * measure count
+   */
+  private int measureColCount;
+  /**
+   * measure count
+   */
+  private int dimColCount;
+  /**
+   * measure count
+   */
+  private int complexDimColCount;
+  /**
+   * fileBufferSize
+   */
+  private int fileBufferSize;
+  /**
+   * numberOfIntermediateFileToBeMerged
+   */
+  private int numberOfIntermediateFileToBeMerged;
+  /**
+   * fileWriteBufferSize
+   */
+  private int fileWriteBufferSize;
+  /**
+   * observer
+   */
+  private SortObserver observer;
+  /**
+   * sortTempFileNoOFRecordsInCompression
+   */
+  private int sortTempFileNoOFRecordsInCompression;
+  /**
+   * isSortTempFileCompressionEnabled
+   */
+  private boolean isSortFileCompressionEnabled;
+  /**
+   * prefetch
+   */
+  private boolean prefetch;
+  /**
+   * bufferSize
+   */
+  private int bufferSize;
+
+  private String databaseName;
+
+  private String tableName;
+
+  private DataType[] measureDataType;
+
+  /**
+   * To know how many columns are of high cardinality.
+   */
+  private int noDictionaryCount;
+  /**
+   * partitionID
+   */
+  private String partitionID;
+  /**
+   * Id of the load folder
+   */
+  private String segmentId;
+  /**
+   * task id, each spark task has a unique id
+   */
+  private String taskNo;
+
+  /**
+   * This will tell whether dimension is dictionary or not.
+   */
+  private boolean[] noDictionaryDimnesionColumn;
+
+  private boolean[] noDictionarySortColumn;
+
+  private int numberOfSortColumns;
+
+  private int numberOfNoDictSortColumns;
+
+  private int numberOfCores;
+
+  private int batchSortSizeinMb;
+
+  public SortParameters getCopy() {
+    SortParameters parameters = new SortParameters();
+    parameters.tempFileLocation = tempFileLocation;
+    parameters.sortBufferSize = sortBufferSize;
+    parameters.measureColCount = measureColCount;
+    parameters.dimColCount = dimColCount;
+    parameters.complexDimColCount = complexDimColCount;
+    parameters.fileBufferSize = fileBufferSize;
+    parameters.numberOfIntermediateFileToBeMerged = numberOfIntermediateFileToBeMerged;
+    parameters.fileWriteBufferSize = fileWriteBufferSize;
+    parameters.observer = observer;
+    parameters.sortTempFileNoOFRecordsInCompression = sortTempFileNoOFRecordsInCompression;
+    parameters.isSortFileCompressionEnabled = isSortFileCompressionEnabled;
+    parameters.prefetch = prefetch;
+    parameters.bufferSize = bufferSize;
+    parameters.databaseName = databaseName;
+    parameters.tableName = tableName;
+    parameters.measureDataType = measureDataType;
+    parameters.noDictionaryCount = noDictionaryCount;
+    parameters.partitionID = partitionID;
+    parameters.segmentId = segmentId;
+    parameters.taskNo = taskNo;
+    parameters.noDictionaryDimnesionColumn = noDictionaryDimnesionColumn;
+    parameters.noDictionarySortColumn = noDictionarySortColumn;
+    parameters.numberOfSortColumns = numberOfSortColumns;
+    parameters.numberOfNoDictSortColumns = numberOfNoDictSortColumns;
+    parameters.numberOfCores = numberOfCores;
+    parameters.batchSortSizeinMb = batchSortSizeinMb;
+    return parameters;
+  }
+
+  public String[] getTempFileLocation() {
+    return tempFileLocation;
+  }
+
+  public void setTempFileLocation(String[] tempFileLocation) {
+    this.tempFileLocation = tempFileLocation;
+  }
+
+  public int getSortBufferSize() {
+    return sortBufferSize;
+  }
+
+  public void setSortBufferSize(int sortBufferSize) {
+    this.sortBufferSize = sortBufferSize;
+  }
+
+  public int getMeasureColCount() {
+    return measureColCount;
+  }
+
+  public void setMeasureColCount(int measureColCount) {
+    this.measureColCount = measureColCount;
+  }
+
+  public int getDimColCount() {
+    return dimColCount;
+  }
+
+  public void setDimColCount(int dimColCount) {
+    this.dimColCount = dimColCount;
+  }
+
+  public int getComplexDimColCount() {
+    return complexDimColCount;
+  }
+
+  public void setComplexDimColCount(int complexDimColCount) {
+    this.complexDimColCount = complexDimColCount;
+  }
+
+  public int getFileBufferSize() {
+    return fileBufferSize;
+  }
+
+  public void setFileBufferSize(int fileBufferSize) {
+    this.fileBufferSize = fileBufferSize;
+  }
+
+  public int getNumberOfIntermediateFileToBeMerged() {
+    return numberOfIntermediateFileToBeMerged;
+  }
+
+  public void setNumberOfIntermediateFileToBeMerged(int numberOfIntermediateFileToBeMerged) {
+    this.numberOfIntermediateFileToBeMerged = numberOfIntermediateFileToBeMerged;
+  }
+
+  public int getFileWriteBufferSize() {
+    return fileWriteBufferSize;
+  }
+
+  public void setFileWriteBufferSize(int fileWriteBufferSize) {
+    this.fileWriteBufferSize = fileWriteBufferSize;
+  }
+
+  public SortObserver getObserver() {
+    return observer;
+  }
+
+  public void setObserver(SortObserver observer) {
+    this.observer = observer;
+  }
+
+  public int getSortTempFileNoOFRecordsInCompression() {
+    return sortTempFileNoOFRecordsInCompression;
+  }
+
+  public void setSortTempFileNoOFRecordsInCompression(int sortTempFileNoOFRecordsInCompression) {
+    this.sortTempFileNoOFRecordsInCompression = sortTempFileNoOFRecordsInCompression;
+  }
+
+  public boolean isSortFileCompressionEnabled() {
+    return isSortFileCompressionEnabled;
+  }
+
+  public void setSortFileCompressionEnabled(boolean sortFileCompressionEnabled) {
+    isSortFileCompressionEnabled = sortFileCompressionEnabled;
+  }
+
+  public boolean isPrefetch() {
+    return prefetch;
+  }
+
+  public void setPrefetch(boolean prefetch) {
+    this.prefetch = prefetch;
+  }
+
+  public int getBufferSize() {
+    return bufferSize;
+  }
+
+  public void setBufferSize(int bufferSize) {
+    this.bufferSize = bufferSize;
+  }
+
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  public void setDatabaseName(String databaseName) {
+    this.databaseName = databaseName;
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  public DataType[] getMeasureDataType() {
+    return measureDataType;
+  }
+
+  public void setMeasureDataType(DataType[] measureDataType) {
+    this.measureDataType = measureDataType;
+  }
+
+  public int getNoDictionaryCount() {
+    return noDictionaryCount;
+  }
+
+  public void setNoDictionaryCount(int noDictionaryCount) {
+    this.noDictionaryCount = noDictionaryCount;
+  }
+
+  public String getPartitionID() {
+    return partitionID;
+  }
+
+  public void setPartitionID(String partitionID) {
+    this.partitionID = partitionID;
+  }
+
+  public String getSegmentId() {
+    return segmentId;
+  }
+
+  public void setSegmentId(String segmentId) {
+    this.segmentId = segmentId;
+  }
+
+  public String getTaskNo() {
+    return taskNo;
+  }
+
+  public void setTaskNo(String taskNo) {
+    this.taskNo = taskNo;
+  }
+
+  public boolean[] getNoDictionaryDimnesionColumn() {
+    return noDictionaryDimnesionColumn;
+  }
+
+  public void setNoDictionaryDimnesionColumn(boolean[] noDictionaryDimnesionColumn) {
+    this.noDictionaryDimnesionColumn = noDictionaryDimnesionColumn;
+  }
+
+  public int getNumberOfCores() {
+    return numberOfCores;
+  }
+
+  public void setNumberOfCores(int numberOfCores) {
+    this.numberOfCores = numberOfCores;
+  }
+
+  public int getNumberOfSortColumns() {
+    return numberOfSortColumns;
+  }
+
+  public void setNumberOfSortColumns(int numberOfSortColumns) {
+    this.numberOfSortColumns = Math.min(numberOfSortColumns, this.dimColCount);
+  }
+
+  public boolean[] getNoDictionarySortColumn() {
+    return noDictionarySortColumn;
+  }
+
+  public void setNoDictionarySortColumn(boolean[] noDictionarySortColumn) {
+    this.noDictionarySortColumn = noDictionarySortColumn;
+  }
+
+  public int getNumberOfNoDictSortColumns() {
+    return numberOfNoDictSortColumns;
+  }
+
+  public void setNumberOfNoDictSortColumns(int numberOfNoDictSortColumns) {
+    this.numberOfNoDictSortColumns = Math.min(numberOfNoDictSortColumns, noDictionaryCount);
+  }
+
+  public int getBatchSortSizeinMb() {
+    return batchSortSizeinMb;
+  }
+
+  public void setBatchSortSizeinMb(int batchSortSizeinMb) {
+    this.batchSortSizeinMb = batchSortSizeinMb;
+  }
+
+  public static SortParameters createSortParameters(CarbonDataLoadConfiguration configuration) {
+    SortParameters parameters = new SortParameters();
+    CarbonTableIdentifier tableIdentifier =
+        configuration.getTableIdentifier().getCarbonTableIdentifier();
+    CarbonProperties carbonProperties = CarbonProperties.getInstance();
+    parameters.setDatabaseName(tableIdentifier.getDatabaseName());
+    parameters.setTableName(tableIdentifier.getTableName());
+    parameters.setPartitionID(configuration.getPartitionId());
+    parameters.setSegmentId(configuration.getSegmentId());
+    parameters.setTaskNo(configuration.getTaskNo());
+    parameters.setMeasureColCount(configuration.getMeasureCount());
+    parameters.setDimColCount(
+        configuration.getDimensionCount() - configuration.getComplexColumnCount());
+    parameters.setNoDictionaryCount(configuration.getNoDictionaryCount());
+    parameters.setComplexDimColCount(configuration.getComplexColumnCount());
+    parameters.setNoDictionaryDimnesionColumn(
+        CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields()));
+    parameters.setBatchSortSizeinMb(CarbonDataProcessorUtil.getBatchSortSizeinMb(configuration));
+
+    parameters.setNumberOfSortColumns(configuration.getNumberOfSortColumns());
+    parameters.setNumberOfNoDictSortColumns(configuration.getNumberOfNoDictSortColumns());
+    setNoDictionarySortColumnMapping(parameters);
+    parameters.setObserver(new SortObserver());
+    // get sort buffer size
+    parameters.setSortBufferSize(Integer.parseInt(carbonProperties
+        .getProperty(CarbonCommonConstants.SORT_SIZE,
+            CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL)));
+    LOGGER.info("Sort size for table: " + parameters.getSortBufferSize());
+    // set number of intermedaite file to merge
+    parameters.setNumberOfIntermediateFileToBeMerged(Integer.parseInt(carbonProperties
+        .getProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT,
+            CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE)));
+
+    LOGGER.info("Number of intermediate file to be merged: " + parameters
+        .getNumberOfIntermediateFileToBeMerged());
+
+    // get file buffer size
+    parameters.setFileBufferSize(CarbonDataProcessorUtil
+        .getFileBufferSize(parameters.getNumberOfIntermediateFileToBeMerged(), carbonProperties,
+            CarbonCommonConstants.CONSTANT_SIZE_TEN));
+
+    LOGGER.info("File Buffer Size: " + parameters.getFileBufferSize());
+
+    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
+        .getLocalDataFolderLocation(tableIdentifier.getDatabaseName(),
+            tableIdentifier.getTableName(), configuration.getTaskNo(),
+            configuration.getPartitionId(), configuration.getSegmentId(), false, false);
+    String[] sortTempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
+        File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
+
+    parameters.setTempFileLocation(sortTempDirs);
+    LOGGER.info("temp file location: " + StringUtils.join(parameters.getTempFileLocation(), ","));
+
+    int numberOfCores;
+    try {
+      numberOfCores = Integer.parseInt(carbonProperties
+          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
+              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
+      numberOfCores = numberOfCores / 2;
+    } catch (NumberFormatException exc) {
+      numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
+    }
+    parameters.setNumberOfCores(numberOfCores > 0 ? numberOfCores : 1);
+
+    parameters.setFileWriteBufferSize(Integer.parseInt(carbonProperties
+        .getProperty(CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE,
+            CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE)));
+
+    parameters.setSortFileCompressionEnabled(Boolean.parseBoolean(carbonProperties
+        .getProperty(CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED,
+            CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED_DEFAULTVALUE)));
+
+    int sortTempFileNoOFRecordsInCompression;
+    try {
+      sortTempFileNoOFRecordsInCompression = Integer.parseInt(carbonProperties
+          .getProperty(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION,
+              CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE));
+      if (sortTempFileNoOFRecordsInCompression < 1) {
+        LOGGER.error("Invalid value for: "
+            + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
+            + ":Only Positive Integer value(greater than zero) is allowed.Default value will "
+            + "be used");
+
+        sortTempFileNoOFRecordsInCompression = Integer.parseInt(
+            CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
+      }
+    } catch (NumberFormatException e) {
+      LOGGER.error(
+          "Invalid value for: " + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
+              + ", only Positive Integer value is allowed. Default value will be used");
+
+      sortTempFileNoOFRecordsInCompression = Integer
+          .parseInt(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
+    }
+    parameters.setSortTempFileNoOFRecordsInCompression(sortTempFileNoOFRecordsInCompression);
+
+    if (parameters.isSortFileCompressionEnabled()) {
+      LOGGER.info("Compression will be used for writing the sort temp File");
+    }
+
+    parameters.setPrefetch(CarbonCommonConstants.CARBON_PREFETCH_IN_MERGE_VALUE);
+    parameters.setBufferSize(Integer.parseInt(carbonProperties.getProperty(
+        CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
+        CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT)));
+
+    DataType[] measureDataType = configuration.getMeasureDataType();
+    parameters.setMeasureDataType(measureDataType);
+    return parameters;
+  }
+
+  /**
+   * this method will set the boolean mapping for no dictionary sort columns
+   *
+   * @param parameters
+   */
+  private static void setNoDictionarySortColumnMapping(SortParameters parameters) {
+    if (parameters.getNumberOfSortColumns() == parameters.getNoDictionaryDimnesionColumn().length) {
+      parameters.setNoDictionarySortColumn(parameters.getNoDictionaryDimnesionColumn());
+    } else {
+      boolean[] noDictionarySortColumnTemp = new boolean[parameters.getNumberOfSortColumns()];
+      System
+          .arraycopy(parameters.getNoDictionaryDimnesionColumn(), 0, noDictionarySortColumnTemp, 0,
+              parameters.getNumberOfSortColumns());
+      parameters.setNoDictionarySortColumn(noDictionarySortColumnTemp);
+    }
+  }
+
+  public static SortParameters createSortParameters(CarbonTable carbonTable, String databaseName,
+      String tableName, int dimColCount, int complexDimColCount, int measureColCount,
+      int noDictionaryCount, String partitionID, 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.setSegmentId(segmentId);
+    parameters.setTaskNo(taskNo);
+    parameters.setMeasureColCount(measureColCount);
+    parameters.setDimColCount(dimColCount - complexDimColCount);
+    parameters.setNumberOfSortColumns(carbonTable.getNumberOfSortColumns());
+    parameters.setNoDictionaryCount(noDictionaryCount);
+    parameters.setNumberOfNoDictSortColumns(carbonTable.getNumberOfNoDictSortColumns());
+    parameters.setComplexDimColCount(complexDimColCount);
+    parameters.setNoDictionaryDimnesionColumn(noDictionaryColMaping);
+    parameters.setObserver(new SortObserver());
+    // get sort buffer size
+    parameters.setSortBufferSize(Integer.parseInt(carbonProperties
+        .getProperty(CarbonCommonConstants.SORT_SIZE,
+            CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL)));
+    LOGGER.info("Sort size for table: " + parameters.getSortBufferSize());
+    // set number of intermedaite file to merge
+    parameters.setNumberOfIntermediateFileToBeMerged(Integer.parseInt(carbonProperties
+        .getProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT,
+            CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE)));
+
+    LOGGER.info("Number of intermediate file to be merged: " + parameters
+        .getNumberOfIntermediateFileToBeMerged());
+
+    // get file buffer size
+    parameters.setFileBufferSize(CarbonDataProcessorUtil
+        .getFileBufferSize(parameters.getNumberOfIntermediateFileToBeMerged(), carbonProperties,
+            CarbonCommonConstants.CONSTANT_SIZE_TEN));
+
+    LOGGER.info("File Buffer Size: " + parameters.getFileBufferSize());
+
+    String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
+        .getLocalDataFolderLocation(databaseName, tableName, taskNo, partitionID, segmentId,
+            isCompactionFlow, false);
+    String[] sortTempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
+        File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
+    parameters.setTempFileLocation(sortTempDirs);
+    LOGGER.info("temp file location: " + StringUtils.join(parameters.getTempFileLocation(), ","));
+
+    int numberOfCores;
+    try {
+      numberOfCores = Integer.parseInt(carbonProperties
+          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
+              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
+      numberOfCores = numberOfCores / 2;
+    } catch (NumberFormatException exc) {
+      numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
+    }
+    parameters.setNumberOfCores(numberOfCores > 0 ? numberOfCores : 1);
+
+    parameters.setFileWriteBufferSize(Integer.parseInt(carbonProperties
+        .getProperty(CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE,
+            CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE)));
+
+    parameters.setSortFileCompressionEnabled(Boolean.parseBoolean(carbonProperties
+        .getProperty(CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED,
+            CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED_DEFAULTVALUE)));
+
+    int sortTempFileNoOFRecordsInCompression;
+    try {
+      sortTempFileNoOFRecordsInCompression = Integer.parseInt(carbonProperties
+          .getProperty(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION,
+              CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE));
+      if (sortTempFileNoOFRecordsInCompression < 1) {
+        LOGGER.error("Invalid value for: "
+            + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
+            + ":Only Positive Integer value(greater than zero) is allowed.Default value will "
+            + "be used");
+
+        sortTempFileNoOFRecordsInCompression = Integer.parseInt(
+            CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
+      }
+    } catch (NumberFormatException e) {
+      LOGGER.error(
+          "Invalid value for: " + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
+              + ", only Positive Integer value is allowed. Default value will be used");
+
+      sortTempFileNoOFRecordsInCompression = Integer
+          .parseInt(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
+    }
+    parameters.setSortTempFileNoOFRecordsInCompression(sortTempFileNoOFRecordsInCompression);
+
+    if (parameters.isSortFileCompressionEnabled()) {
+      LOGGER.info("Compression will be used for writing the sort temp File");
+    }
+
+    parameters.setPrefetch(CarbonCommonConstants. CARBON_PREFETCH_IN_MERGE_VALUE);
+    parameters.setBufferSize(Integer.parseInt(carbonProperties.getProperty(
+        CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
+        CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT)));
+
+    DataType[] type = CarbonDataProcessorUtil
+        .getMeasureDataType(parameters.getMeasureColCount(), parameters.getDatabaseName(),
+            parameters.getTableName());
+    parameters.setMeasureDataType(type);
+    setNoDictionarySortColumnMapping(parameters);
+    return parameters;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..c4b0b31
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkHolder.java
@@ -0,0 +1,522 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.BufferedInputStream;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+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.metadata.datatype.DataType;
+import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
+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.NonDictionaryUtil;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHolder> {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(SortTempFileChunkHolder.class.getName());
+
+  /**
+   * temp file
+   */
+  private File tempFile;
+
+  /**
+   * read stream
+   */
+  private DataInputStream stream;
+
+  /**
+   * entry count
+   */
+  private int entryCount;
+
+  /**
+   * number record read
+   */
+  private int numberOfObjectRead;
+
+  /**
+   * return row
+   */
+  private Object[] returnRow;
+
+  /**
+   * number of measures
+   */
+  private int measureCount;
+
+  /**
+   * number of dimensionCount
+   */
+  private int dimensionCount;
+
+  /**
+   * number of complexDimensionCount
+   */
+  private int complexDimensionCount;
+
+  /**
+   * fileBufferSize for file reader stream size
+   */
+  private int fileBufferSize;
+
+  private Object[][] currentBuffer;
+
+  private Object[][] backupBuffer;
+
+  private boolean isBackupFilled;
+
+  private boolean prefetch;
+
+  private int bufferSize;
+
+  private int bufferRowCounter;
+
+  private ExecutorService executorService;
+
+  private Future<Void> submit;
+
+  private int prefetchRecordsProceesed;
+
+  /**
+   * sortTempFileNoOFRecordsInCompression
+   */
+  private int sortTempFileNoOFRecordsInCompression;
+
+  /**
+   * isSortTempFileCompressionEnabled
+   */
+  private boolean isSortTempFileCompressionEnabled;
+
+  /**
+   * totalRecordFetch
+   */
+  private int totalRecordFetch;
+
+  private int noDictionaryCount;
+
+  private DataType[] aggType;
+
+  /**
+   * to store whether dimension is of dictionary type or not
+   */
+  private boolean[] isNoDictionaryDimensionColumn;
+
+  /**
+   * to store whether sort column is of dictionary type or not
+   */
+  private boolean[] isNoDictionarySortColumn;
+
+  /**
+   * Constructor to initialize
+   *
+   * @param tempFile
+   * @param dimensionCount
+   * @param complexDimensionCount
+   * @param measureCount
+   * @param fileBufferSize
+   * @param noDictionaryCount
+   * @param aggType
+   * @param isNoDictionaryDimensionColumn
+   */
+  public SortTempFileChunkHolder(File tempFile, int dimensionCount, int complexDimensionCount,
+      int measureCount, int fileBufferSize, int noDictionaryCount, DataType[] aggType,
+      boolean[] isNoDictionaryDimensionColumn, boolean[] isNoDictionarySortColumn) {
+    // set temp file
+    this.tempFile = tempFile;
+
+    // set measure and dimension count
+    this.measureCount = measureCount;
+    this.dimensionCount = dimensionCount;
+    this.complexDimensionCount = complexDimensionCount;
+
+    this.noDictionaryCount = noDictionaryCount;
+    // set mdkey length
+    this.fileBufferSize = fileBufferSize;
+    this.executorService = Executors.newFixedThreadPool(1);
+    this.aggType = aggType;
+
+    this.isNoDictionaryDimensionColumn = isNoDictionaryDimensionColumn;
+    this.isNoDictionarySortColumn = isNoDictionarySortColumn;
+  }
+
+  /**
+   * This method will be used to initialize
+   *
+   * @throws CarbonSortKeyAndGroupByException problem while initializing
+   */
+  public void initialize() throws CarbonSortKeyAndGroupByException {
+    prefetch = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_PREFETCH,
+            CarbonCommonConstants.CARBON_MERGE_SORT_PREFETCH_DEFAULT));
+    bufferSize = Integer.parseInt(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
+            CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT));
+    this.isSortTempFileCompressionEnabled = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED,
+            CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED_DEFAULTVALUE));
+    if (this.isSortTempFileCompressionEnabled) {
+      LOGGER.info("Compression was used while writing the sortTempFile");
+    }
+
+    try {
+      this.sortTempFileNoOFRecordsInCompression = Integer.parseInt(CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION,
+              CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE));
+      if (this.sortTempFileNoOFRecordsInCompression < 1) {
+        LOGGER.error("Invalid value for: "
+            + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
+            + ": Only Positive Integer value(greater than zero) is allowed.Default value will"
+            + " be used");
+
+        this.sortTempFileNoOFRecordsInCompression = Integer.parseInt(
+            CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
+      }
+    } catch (NumberFormatException e) {
+      LOGGER.error(
+          "Invalid value for: " + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
+              + ", only Positive Integer value is allowed.Default value will be used");
+      this.sortTempFileNoOFRecordsInCompression = Integer
+          .parseInt(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
+    }
+
+    initialise();
+  }
+
+  private void initialise() throws CarbonSortKeyAndGroupByException {
+    try {
+      if (isSortTempFileCompressionEnabled) {
+        this.bufferSize = sortTempFileNoOFRecordsInCompression;
+      }
+      stream = new DataInputStream(
+          new BufferedInputStream(new FileInputStream(tempFile), this.fileBufferSize));
+      this.entryCount = stream.readInt();
+      if (prefetch) {
+        new DataFetcher(false).call();
+        totalRecordFetch += currentBuffer.length;
+        if (totalRecordFetch < this.entryCount) {
+          submit = executorService.submit(new DataFetcher(true));
+        }
+      } else {
+        if (isSortTempFileCompressionEnabled) {
+          new DataFetcher(false).call();
+        }
+      }
+
+    } catch (FileNotFoundException e) {
+      LOGGER.error(e);
+      throw new CarbonSortKeyAndGroupByException(tempFile + " No Found", e);
+    } catch (IOException e) {
+      LOGGER.error(e);
+      throw new CarbonSortKeyAndGroupByException(tempFile + " No Found", e);
+    } catch (Exception e) {
+      LOGGER.error(e);
+      throw new CarbonSortKeyAndGroupByException(tempFile + " Problem while reading", e);
+    }
+  }
+
+  /**
+   * This method will be used to read new row from file
+   *
+   * @throws CarbonSortKeyAndGroupByException problem while reading
+   */
+  public void readRow() throws CarbonSortKeyAndGroupByException {
+    if (prefetch) {
+      fillDataForPrefetch();
+    } else if (isSortTempFileCompressionEnabled) {
+      if (bufferRowCounter >= bufferSize) {
+        try {
+          new DataFetcher(false).call();
+          bufferRowCounter = 0;
+        } catch (Exception e) {
+          LOGGER.error(e);
+          throw new CarbonSortKeyAndGroupByException(tempFile + " Problem while reading", e);
+        }
+
+      }
+      prefetchRecordsProceesed++;
+      returnRow = currentBuffer[bufferRowCounter++];
+    } else {
+      this.returnRow = getRowFromStream();
+    }
+  }
+
+  private void fillDataForPrefetch() {
+    if (bufferRowCounter >= bufferSize) {
+      if (isBackupFilled) {
+        bufferRowCounter = 0;
+        currentBuffer = backupBuffer;
+        totalRecordFetch += currentBuffer.length;
+        isBackupFilled = false;
+        if (totalRecordFetch < this.entryCount) {
+          submit = executorService.submit(new DataFetcher(true));
+        }
+      } else {
+        try {
+          submit.get();
+        } catch (Exception e) {
+          LOGGER.error(e);
+        }
+        bufferRowCounter = 0;
+        currentBuffer = backupBuffer;
+        isBackupFilled = false;
+        totalRecordFetch += currentBuffer.length;
+        if (totalRecordFetch < this.entryCount) {
+          submit = executorService.submit(new DataFetcher(true));
+        }
+      }
+    }
+    prefetchRecordsProceesed++;
+    returnRow = currentBuffer[bufferRowCounter++];
+  }
+
+  /**
+   * Reads row from file
+   * @return Object[]
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  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[this.dimensionCount - this.noDictionaryCount];
+    byte[][] nonDicArray = new byte[this.noDictionaryCount + this.complexDimensionCount][];
+    Object[] measures = new Object[this.measureCount];
+    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 < complexDimensionCount; 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 < this.measureCount; i++) {
+        if (stream.readByte() == 1) {
+          switch (aggType[i]) {
+            case SHORT:
+              measures[index++] = stream.readShort();
+              break;
+            case INT:
+              measures[index++] = stream.readInt();
+              break;
+            case LONG:
+              measures[index++] = stream.readLong();
+              break;
+            case DOUBLE:
+              measures[index++] = stream.readDouble();
+              break;
+            case DECIMAL:
+              int len = stream.readInt();
+              byte[] buff = new byte[len];
+              stream.readFully(buff);
+              measures[index++] = DataTypeUtil.byteToBigDecimal(buff);
+              break;
+            default:
+              throw new IllegalArgumentException("unsupported data type:" + aggType[i]);
+          }
+        } 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);
+    }
+
+    //return out row
+    return holder;
+  }
+
+  /**
+   * below method will be used to get the row
+   *
+   * @return row
+   */
+  public Object[] getRow() {
+    return this.returnRow;
+  }
+
+  /**
+   * below method will be used to check whether any more records are present
+   * in file or not
+   *
+   * @return more row present in file
+   */
+  public boolean hasNext() {
+    if (prefetch || isSortTempFileCompressionEnabled) {
+      return this.prefetchRecordsProceesed < this.entryCount;
+    }
+    return this.numberOfObjectRead < this.entryCount;
+  }
+
+  /**
+   * Below method will be used to close streams
+   */
+  public void closeStream() {
+    CarbonUtil.closeStreams(stream);
+    executorService.shutdown();
+    this.backupBuffer = null;
+    this.currentBuffer = null;
+  }
+
+  /**
+   * This method will number of entries
+   *
+   * @return entryCount
+   */
+  public int getEntryCount() {
+    return entryCount;
+  }
+
+  @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;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+
+    if (!(obj instanceof SortTempFileChunkHolder)) {
+      return false;
+    }
+    SortTempFileChunkHolder o = (SortTempFileChunkHolder) obj;
+
+    return this == o;
+  }
+
+  @Override public int hashCode() {
+    int hash = 0;
+    hash += 31 * measureCount;
+    hash += 31 * dimensionCount;
+    hash += 31 * complexDimensionCount;
+    hash += 31 * noDictionaryCount;
+    hash += tempFile.hashCode();
+    return hash;
+  }
+
+  private final class DataFetcher implements Callable<Void> {
+    private boolean isBackUpFilling;
+
+    private int numberOfRecords;
+
+    private DataFetcher(boolean backUp) {
+      isBackUpFilling = backUp;
+      calculateNumberOfRecordsToBeFetched();
+    }
+
+    private void calculateNumberOfRecordsToBeFetched() {
+      int numberOfRecordsLeftToBeRead = entryCount - totalRecordFetch;
+      numberOfRecords =
+          bufferSize < numberOfRecordsLeftToBeRead ? bufferSize : numberOfRecordsLeftToBeRead;
+    }
+
+    @Override public Void call() throws Exception {
+      try {
+        if (isBackUpFilling) {
+          backupBuffer = prefetchRecordsFromFile(numberOfRecords);
+          isBackupFilled = true;
+        } else {
+          currentBuffer = prefetchRecordsFromFile(numberOfRecords);
+        }
+      } catch (Exception e) {
+        LOGGER.error(e);
+      }
+      return null;
+    }
+
+  }
+
+  /**
+   * This method will read the records from sort temp file and keep it in a buffer
+   *
+   * @param numberOfRecords
+   * @return
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkWriter.java
new file mode 100644
index 0000000..025aef8
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/SortTempFileChunkWriter.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.processing.sort.sortdata;
+
+import java.io.File;
+
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+public class SortTempFileChunkWriter implements TempSortFileWriter {
+  /**
+   * writer
+   */
+  private TempSortFileWriter writer;
+
+  /**
+   * recordPerLeaf
+   */
+  private int recordPerLeaf;
+
+  /**
+   * CarbonCompressedSortTempFileChunkWriter
+   *
+   * @param writer
+   */
+  public SortTempFileChunkWriter(TempSortFileWriter writer, int recordPerLeaf) {
+    this.writer = writer;
+    this.recordPerLeaf = recordPerLeaf;
+  }
+
+  /**
+   * initialize
+   */
+  public void initiaize(File file, int entryCount) throws CarbonSortKeyAndGroupByException {
+    this.writer.initiaize(file, entryCount);
+  }
+
+  /**
+   * finish
+   */
+  public void finish() {
+    this.writer.finish();
+  }
+
+  /**
+   * Below method will be used to write the sort temp file chunk by chunk
+   */
+  public void writeSortTempFile(Object[][] records) throws CarbonSortKeyAndGroupByException {
+    int recordCount = 0;
+    Object[][] tempRecords;
+    while (recordCount < records.length) {
+      if (records.length - recordCount < recordPerLeaf) {
+        recordPerLeaf = records.length - recordCount;
+      }
+      tempRecords = new Object[recordPerLeaf][];
+      System.arraycopy(records, recordCount, tempRecords, 0, recordPerLeaf);
+      recordCount += recordPerLeaf;
+      this.writer.writeSortTempFile(tempRecords);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileReader.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileReader.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileReader.java
new file mode 100644
index 0000000..0de9af7
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileReader.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.processing.sort.sortdata;
+
+public interface TempSortFileReader {
+  /**
+   * below method will be used to close the file holder
+   */
+  void finish();
+
+  /**
+   * Below method will be used to get the row
+   */
+  Object[][] getRow();
+
+  /**
+   * Below method will be used to get the total row count in temp file
+   *
+   * @return
+   */
+  int getEntryCount();
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileWriter.java
new file mode 100644
index 0000000..4e4a8e7
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileWriter.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.processing.sort.sortdata;
+
+import java.io.File;
+
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+
+public interface TempSortFileWriter {
+  /**
+   * Method will be used to initialize
+   *
+   * @param file
+   * @param entryCount
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  void initiaize(File file, int entryCount) throws CarbonSortKeyAndGroupByException;
+
+  /**
+   * Method will be used to finish
+   */
+  void finish();
+
+  /**
+   * Below method will be used to write the sort temp file
+   *
+   * @param records
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  void writeSortTempFile(Object[][] records) throws CarbonSortKeyAndGroupByException;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileWriterFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileWriterFactory.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileWriterFactory.java
new file mode 100644
index 0000000..259ab9f
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TempSortFileWriterFactory.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.processing.sort.sortdata;
+
+public final class TempSortFileWriterFactory {
+  private static final TempSortFileWriterFactory WRITERFACTORY = new TempSortFileWriterFactory();
+
+  private TempSortFileWriterFactory() {
+
+  }
+
+  public static TempSortFileWriterFactory getInstance() {
+    return WRITERFACTORY;
+  }
+
+  public TempSortFileWriter getTempSortFileWriter(boolean isCompressionEnabled, int dimensionCount,
+      int complexDimensionCount, int measureCount, int noDictionaryCount, int writeBufferSize) {
+    if (isCompressionEnabled) {
+      return new CompressedTempSortFileWriter(dimensionCount, complexDimensionCount, measureCount,
+          noDictionaryCount, writeBufferSize);
+    } else {
+      return new UnCompressedTempSortFileWriter(dimensionCount, complexDimensionCount, measureCount,
+          noDictionaryCount, writeBufferSize);
+    }
+  }
+}


[09/50] [abbrv] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
new file mode 100644
index 0000000..897895a
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableCommand.scala
@@ -0,0 +1,520 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.management
+
+import scala.collection.JavaConverters._
+
+import org.apache.commons.lang3.StringUtils
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.execution.command.{DataLoadTableFileMapping, DataProcessCommand, RunnableCommand, UpdateTableModel}
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.util.{CausedBy, FileUtils}
+
+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.datastore.impl.FileFactory
+import org.apache.carbondata.core.dictionary.server.DictionaryServer
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format
+import org.apache.carbondata.processing.constants.TableOptionConstant
+import org.apache.carbondata.processing.etl.DataLoadingException
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
+import org.apache.carbondata.processing.newflow.exception.NoRetryException
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+import org.apache.carbondata.spark.load.ValidateUtil
+import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DictionaryLoadModel}
+import org.apache.carbondata.spark.util.{CommonUtil, GlobalDictionaryUtil}
+
+case class LoadTableCommand(
+    databaseNameOp: Option[String],
+    tableName: String,
+    factPathFromUser: String,
+    dimFilesPath: Seq[DataLoadTableFileMapping],
+    options: scala.collection.immutable.Map[String, String],
+    isOverwriteTable: Boolean,
+    var inputSqlString: String = null,
+    dataFrame: Option[DataFrame] = None,
+    updateModel: Option[UpdateTableModel] = None)
+  extends RunnableCommand with DataProcessCommand {
+
+  private def getFinalOptions(carbonProperty: CarbonProperties):
+  scala.collection.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("escapechar", options.getOrElse("escapechar", "\\"))
+    optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
+    optionsFinal.put("columndict", options.getOrElse("columndict", null))
+    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("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("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
+  }
+
+  private def checkDefaultValue(value: String, default: String) = {
+    if (StringUtils.isEmpty(value)) {
+      default
+    } else {
+      value
+    }
+  }
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    if (dataFrame.isDefined && updateModel.isEmpty) {
+      val rdd = dataFrame.get.rdd
+      if (rdd.partitions == null || rdd.partitions.length == 0) {
+        LOGGER.warn("DataLoading finished. No data was loaded.")
+        return Seq.empty
+      }
+    }
+
+    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
+
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(Option(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+    if (null == relation.tableMeta.carbonTable) {
+      LOGGER.error(s"Data loading failed. table not found: $dbName.$tableName")
+      LOGGER.audit(s"Data loading failed. table not found: $dbName.$tableName")
+      sys.error(s"Data loading failed. table not found: $dbName.$tableName")
+    }
+
+    val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
+    carbonProperty.addProperty("zookeeper.enable.lock", "false")
+    val optionsFinal = getFinalOptions(carbonProperty)
+
+    val tableProperties = relation.tableMeta.carbonTable.getTableInfo
+      .getFactTable.getTableProperties
+
+    optionsFinal.put("sort_scope", tableProperties.getOrDefault("sort_scope",
+      carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
+        carbonProperty.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
+          CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))))
+
+    try {
+      val factPath = if (dataFrame.isDefined) {
+        ""
+      } else {
+        FileUtils.getPaths(
+          CarbonUtil.checkAndAppendHDFSUrl(factPathFromUser))
+      }
+      val carbonLoadModel = new CarbonLoadModel()
+      carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
+      carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
+      carbonLoadModel.setStorePath(relation.tableMeta.carbonTable.getStorePath)
+
+      val table = relation.tableMeta.carbonTable
+      carbonLoadModel.setTableName(table.getFactTableName)
+      val dataLoadSchema = new CarbonDataLoadSchema(table)
+      // Need to fill dimension relation
+      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+
+      val partitionLocation = relation.tableMeta.storePath + "/partition/" +
+                              relation.tableMeta.carbonTableIdentifier.getDatabaseName + "/" +
+                              relation.tableMeta.carbonTableIdentifier.getTableName + "/"
+      val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean
+      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")
+      val bad_record_path = optionsFinal("bad_record_path")
+      val global_sort_partitions = optionsFinal("global_sort_partitions")
+      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.validateDateFormat(dateFormat, table, tableName)
+      ValidateUtil.validateSortScope(table, sort_scope)
+
+      if (bad_records_logger_enable.toBoolean ||
+          LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
+        if (!CarbonUtil.isValidBadStorePath(bad_record_path)) {
+          sys.error("Invalid bad records location.")
+        }
+      }
+      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.getFactTableName)
+              .asScala.map(_.getColName).mkString(",")
+          }
+        }
+      }
+
+      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)
+      carbonLoadModel
+        .setIsEmptyDataBadRecord(
+          DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," +
+          optionsFinal("is_empty_data_bad_record"))
+      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)) {
+        sys.error(s"Field Delimiter & 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)
+
+      val partitionStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
+      try {
+        // 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)")
+        carbonLoadModel.setFactFilePath(factPath)
+        carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimeter))
+        carbonLoadModel.setCsvHeader(fileHeader)
+        carbonLoadModel.setColDictFilePath(column_dict)
+        carbonLoadModel.setDirectLoad(true)
+        carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
+        val validatedMaxColumns = CommonUtil.validateMaxColumns(carbonLoadModel.getCsvHeaderColumns,
+          optionsFinal("maxcolumns"))
+        carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
+        GlobalDictionaryUtil.updateTableMetadataFunc = updateTableMetadata
+        val storePath = relation.tableMeta.storePath
+        // add the start entry for the new load in the table status file
+        if (updateModel.isEmpty) {
+          CommonUtil.
+            readAndUpdateLoadProgressInTableMeta(carbonLoadModel, storePath, isOverwriteTable)
+        }
+        if (isOverwriteTable) {
+          LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
+        }
+        if (null == carbonLoadModel.getLoadMetadataDetails) {
+          CommonUtil.readLoadMetadataDetails(carbonLoadModel)
+        }
+        if (carbonLoadModel.getLoadMetadataDetails.isEmpty && carbonLoadModel.getUseOnePass &&
+            StringUtils.isEmpty(column_dict) && StringUtils.isEmpty(all_dictionary_path)) {
+          LOGGER.info(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
+          LOGGER.audit(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
+          carbonLoadModel.setUseOnePass(false)
+        }
+        // Create table and metadata folders if not exist
+        val carbonTablePath = CarbonStorePath
+          .getCarbonTablePath(storePath, table.getCarbonTableIdentifier)
+        val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
+        val fileType = FileFactory.getFileType(metadataDirectoryPath)
+        if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
+          FileFactory.mkdirs(metadataDirectoryPath, fileType)
+        }
+        if (carbonLoadModel.getUseOnePass) {
+          val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+          val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+            .getCarbonTableIdentifier
+          val carbonTablePath = CarbonStorePath
+            .getCarbonTablePath(storePath, carbonTableIdentifier)
+          val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
+          val dimensions = carbonTable.getDimensionByTableName(
+            carbonTable.getFactTableName).asScala.toArray
+          val colDictFilePath = carbonLoadModel.getColDictFilePath
+          if (!StringUtils.isEmpty(colDictFilePath)) {
+            carbonLoadModel.initPredefDictMap()
+            // generate predefined dictionary
+            GlobalDictionaryUtil
+              .generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
+                dimensions, carbonLoadModel, sparkSession.sqlContext, storePath, dictFolderPath)
+          }
+          if (!StringUtils.isEmpty(all_dictionary_path)) {
+            carbonLoadModel.initPredefDictMap()
+            GlobalDictionaryUtil
+              .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
+                carbonLoadModel,
+                storePath,
+                carbonTableIdentifier,
+                dictFolderPath,
+                dimensions,
+                all_dictionary_path)
+          }
+          // dictionaryServerClient dictionary generator
+          val dictionaryServerPort = carbonProperty
+            .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
+              CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
+          val sparkDriverHost = sparkSession.sqlContext.sparkContext.
+            getConf.get("spark.driver.host")
+          carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
+          // start dictionary server when use one pass load and dimension with DICTIONARY
+          // encoding is present.
+          val allDimensions = table.getAllDimensions.asScala.toList
+          val createDictionary = allDimensions.exists {
+            carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
+                               !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
+          }
+          val server: Option[DictionaryServer] = if (createDictionary) {
+            val dictionaryServer = DictionaryServer
+              .getInstance(dictionaryServerPort.toInt, carbonTable)
+            carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
+            sparkSession.sparkContext.addSparkListener(new SparkListener() {
+              override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
+                dictionaryServer.shutdown()
+              }
+            })
+            Some(dictionaryServer)
+          } else {
+            None
+          }
+          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+            carbonLoadModel,
+            relation.tableMeta.storePath,
+            columnar,
+            partitionStatus,
+            server,
+            isOverwriteTable,
+            dataFrame,
+            updateModel)
+        } else {
+          val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
+            val fields = dataFrame.get.schema.fields
+            import org.apache.spark.sql.functions.udf
+            // extracting only segment from tupleId
+            val getSegIdUDF = udf((tupleId: String) =>
+              CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID))
+            // getting all fields except tupleId field as it is not required in the value
+            var otherFields = fields.toSeq
+              .filter(field => !field.name
+                .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
+              .map(field => new Column(field.name))
+
+            // extract tupleId field which will be used as a key
+            val segIdColumn = getSegIdUDF(new Column(UnresolvedAttribute
+              .quotedString(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))).
+              as(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID)
+            // use dataFrameWithoutTupleId as dictionaryDataFrame
+            val dataFrameWithoutTupleId = dataFrame.get.select(otherFields: _*)
+            otherFields = otherFields :+ segIdColumn
+            // use dataFrameWithTupleId as loadDataFrame
+            val dataFrameWithTupleId = dataFrame.get.select(otherFields: _*)
+            (Some(dataFrameWithoutTupleId), Some(dataFrameWithTupleId))
+          } else {
+            (dataFrame, dataFrame)
+          }
+
+          GlobalDictionaryUtil.generateGlobalDictionary(
+            sparkSession.sqlContext,
+            carbonLoadModel,
+            relation.tableMeta.storePath,
+            dictionaryDataFrame)
+          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
+            carbonLoadModel,
+            relation.tableMeta.storePath,
+            columnar,
+            partitionStatus,
+            None,
+            isOverwriteTable,
+            loadDataFrame,
+            updateModel)
+        }
+      } catch {
+        case CausedBy(ex: NoRetryException) =>
+          LOGGER.error(ex, s"Dataload failure for $dbName.$tableName")
+          throw new RuntimeException(s"Dataload failure for $dbName.$tableName, ${ex.getMessage}")
+        case ex: Exception =>
+          LOGGER.error(ex)
+          LOGGER.audit(s"Dataload failure for $dbName.$tableName. Please check the logs")
+          throw ex
+      } finally {
+        // Once the data load is successful delete the unwanted partition files
+        try {
+          val fileType = FileFactory.getFileType(partitionLocation)
+          if (FileFactory.isFileExist(partitionLocation, fileType)) {
+            val file = FileFactory
+              .getCarbonFile(partitionLocation, fileType)
+            CarbonUtil.deleteFoldersAndFiles(file)
+          }
+        } catch {
+          case ex: Exception =>
+            LOGGER.error(ex)
+            LOGGER.audit(s"Dataload failure for $dbName.$tableName. " +
+                         "Problem deleting the partition folder")
+            throw ex
+        }
+
+      }
+    } catch {
+      case dle: DataLoadingException =>
+        LOGGER.audit(s"Dataload failed for $dbName.$tableName. " + dle.getMessage)
+        throw dle
+      case mce: MalformedCarbonCommandException =>
+        LOGGER.audit(s"Dataload failed for $dbName.$tableName. " + mce.getMessage)
+        throw mce
+    }
+    Seq.empty
+  }
+
+  private def updateTableMetadata(
+      carbonLoadModel: CarbonLoadModel,
+      sqlContext: SQLContext,
+      model: DictionaryLoadModel,
+      noDictDimension: Array[CarbonDimension]): Unit = {
+    val sparkSession = sqlContext.sparkSession
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(model.hdfsLocation,
+      model.table)
+
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    // read TableInfo
+    val tableInfo: format.TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+
+    // modify TableInfo
+    val columns = tableInfo.getFact_table.getTable_columns
+    for (i <- 0 until columns.size) {
+      if (noDictDimension.exists(x => columns.get(i).getColumn_id.equals(x.getColumnId))) {
+        columns.get(i).encoders.remove(org.apache.carbondata.format.Encoding.DICTIONARY)
+      }
+    }
+    val entry = tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
+    entry.setTime_stamp(System.currentTimeMillis())
+
+    // write TableInfo
+    metastore.updateTableSchema(carbonTablePath.getCarbonTableIdentifier,
+      carbonTablePath.getCarbonTableIdentifier,
+      tableInfo, entry, carbonTablePath.getPath)(sparkSession)
+
+    // update the schema modified time
+    metastore.updateAndTouchSchemasUpdatedTime(model.hdfsLocation)
+
+    // update CarbonDataLoadSchema
+    val carbonTable = metastore.lookupRelation(Option(model.table.getDatabaseName),
+      model.table.getTableName)(sqlContext.sparkSession).asInstanceOf[CarbonRelation].tableMeta
+      .carbonTable
+    carbonLoadModel.setCarbonDataLoadSchema(new CarbonDataLoadSchema(carbonTable))
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
new file mode 100644
index 0000000..02c7023
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
@@ -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.spark.sql.execution.command.mutation
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command.ExecutionErrors
+import org.apache.spark.sql.hive.CarbonRelation
+
+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.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.SegmentUpdateStatusManager
+import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.writer.CarbonDeleteDeltaWriterImpl
+import org.apache.carbondata.hadoop.CarbonInputFormat
+import org.apache.carbondata.processing.exception.MultipleMatchingException
+import org.apache.carbondata.spark.DeleteDelataResultImpl
+import org.apache.carbondata.spark.load.FailureCauses
+import org.apache.carbondata.spark.util.QueryPlanUtil
+
+object DeleteExecution {
+  val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+
+  def getTableIdentifier(tableIdentifier: Seq[String]): TableIdentifier = {
+    if (tableIdentifier.size > 1) {
+      TableIdentifier(tableIdentifier(1), Some(tableIdentifier(0)))
+    } else {
+      TableIdentifier(tableIdentifier(0), None)
+    }
+  }
+
+  def deleteDeltaExecution(
+      identifier: Seq[String],
+      sparkSession: SparkSession,
+      dataRdd: RDD[Row],
+      timestamp: String, relation: CarbonRelation, isUpdateOperation: Boolean,
+      executorErrors: ExecutionErrors
+  ): Boolean = {
+
+    var res: Array[List[(String, (SegmentUpdateDetails, ExecutionErrors))]] = null
+    val tableName = getTableIdentifier(identifier).table
+    val database = GetDB.getDatabaseName(getTableIdentifier(identifier).database, sparkSession)
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(DeleteExecution.getTableIdentifier(identifier))(sparkSession).
+      asInstanceOf[CarbonRelation]
+
+    val storeLocation = relation.tableMeta.storePath
+    val absoluteTableIdentifier: AbsoluteTableIdentifier = new
+        AbsoluteTableIdentifier(storeLocation,
+          relation.tableMeta.carbonTableIdentifier)
+    val tablePath = CarbonStorePath.getCarbonTablePath(
+      storeLocation,
+      absoluteTableIdentifier.getCarbonTableIdentifier)
+    val factPath = tablePath.getFactDir
+
+    val carbonTable = relation.tableMeta.carbonTable
+    var deleteStatus = true
+    val deleteRdd = if (isUpdateOperation) {
+      val schema =
+        org.apache.spark.sql.types.StructType(Seq(org.apache.spark.sql.types.StructField(
+          CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID,
+          org.apache.spark.sql.types.StringType)))
+      val rdd = dataRdd
+        .map(row => Row(row.get(row.fieldIndex(
+          CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))))
+      sparkSession.createDataFrame(rdd, schema).rdd
+      // sqlContext.createDataFrame(rdd, schema).rdd
+    } else {
+      dataRdd
+    }
+
+    val (carbonInputFormat, job) =
+      QueryPlanUtil.createCarbonInputFormat(absoluteTableIdentifier)
+    CarbonInputFormat.setTableInfo(job.getConfiguration, carbonTable.getTableInfo)
+    val keyRdd = deleteRdd.map({ row =>
+      val tupleId: String = row
+        .getString(row.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
+      val key = CarbonUpdateUtil.getSegmentWithBlockFromTID(tupleId)
+      (key, row)
+    }).groupByKey()
+
+    // if no loads are present then no need to do anything.
+    if (keyRdd.partitions.length == 0) {
+      return true
+    }
+
+    val blockMappingVO = carbonInputFormat.getBlockRowCount(job, absoluteTableIdentifier)
+    val segmentUpdateStatusMngr = new SegmentUpdateStatusManager(absoluteTableIdentifier)
+    CarbonUpdateUtil
+      .createBlockDetailsMap(blockMappingVO, segmentUpdateStatusMngr)
+
+    val rowContRdd =
+      sparkSession.sparkContext.parallelize(
+        blockMappingVO.getCompleteBlockRowDetailVO.asScala.toSeq,
+        keyRdd.partitions.length)
+
+    val rdd = rowContRdd.join(keyRdd)
+    res = rdd.mapPartitionsWithIndex(
+      (index: Int, records: Iterator[((String), (RowCountDetailsVO, Iterable[Row]))]) =>
+        Iterator[List[(String, (SegmentUpdateDetails, ExecutionErrors))]] {
+
+          var result = List[(String, (SegmentUpdateDetails, ExecutionErrors))]()
+          while (records.hasNext) {
+            val ((key), (rowCountDetailsVO, groupedRows)) = records.next
+            result = result ++
+                     deleteDeltaFunc(index,
+                       key,
+                       groupedRows.toIterator,
+                       timestamp,
+                       rowCountDetailsVO)
+          }
+          result
+        }
+    ).collect()
+
+    // if no loads are present then no need to do anything.
+    if (res.isEmpty) {
+      return true
+    }
+
+    // update new status file
+    checkAndUpdateStatusFiles()
+
+    // all or none : update status file, only if complete delete opeartion is successfull.
+    def checkAndUpdateStatusFiles(): Unit = {
+      val blockUpdateDetailsList = new util.ArrayList[SegmentUpdateDetails]()
+      val segmentDetails = new util.HashSet[String]()
+      res.foreach(resultOfSeg => resultOfSeg.foreach(
+        resultOfBlock => {
+          if (resultOfBlock._1.equalsIgnoreCase(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)) {
+            blockUpdateDetailsList.add(resultOfBlock._2._1)
+            segmentDetails.add(resultOfBlock._2._1.getSegmentName)
+            // if this block is invalid then decrement block count in map.
+            if (CarbonUpdateUtil.isBlockInvalid(resultOfBlock._2._1.getStatus)) {
+              CarbonUpdateUtil.decrementDeletedBlockCount(resultOfBlock._2._1,
+                blockMappingVO.getSegmentNumberOfBlockMapping)
+            }
+          }
+          else {
+            deleteStatus = false
+            // In case of failure , clean all related delete delta files
+            CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
+            LOGGER.audit(s"Delete data operation is failed for ${ database }.${ tableName }")
+            val errorMsg =
+              "Delete data operation is failed due to failure in creating delete delta file for " +
+              "segment : " + resultOfBlock._2._1.getSegmentName + " block : " +
+              resultOfBlock._2._1.getBlockName
+            executorErrors.failureCauses = resultOfBlock._2._2.failureCauses
+            executorErrors.errorMsg = resultOfBlock._2._2.errorMsg
+
+            if (executorErrors.failureCauses == FailureCauses.NONE) {
+              executorErrors.failureCauses = FailureCauses.EXECUTOR_FAILURE
+              executorErrors.errorMsg = errorMsg
+            }
+            LOGGER.error(errorMsg)
+            return
+          }
+        }
+      )
+      )
+
+      val listOfSegmentToBeMarkedDeleted = CarbonUpdateUtil
+        .getListOfSegmentsToMarkDeleted(blockMappingVO.getSegmentNumberOfBlockMapping)
+
+
+
+      // this is delete flow so no need of putting timestamp in the status file.
+      if (CarbonUpdateUtil
+            .updateSegmentStatus(blockUpdateDetailsList, carbonTable, timestamp, false) &&
+          CarbonUpdateUtil
+            .updateTableMetadataStatus(segmentDetails,
+              carbonTable,
+              timestamp,
+              !isUpdateOperation,
+              listOfSegmentToBeMarkedDeleted)
+      ) {
+        LOGGER.info(s"Delete data operation is successful for ${ database }.${ tableName }")
+        LOGGER.audit(s"Delete data operation is successful for ${ database }.${ tableName }")
+      }
+      else {
+        // In case of failure , clean all related delete delta files
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
+
+        val errorMessage = "Delete data operation is failed due to failure " +
+                           "in table status updation."
+        LOGGER.audit(s"Delete data operation is failed for ${ database }.${ tableName }")
+        LOGGER.error("Delete data operation is failed due to failure in table status updation.")
+        executorErrors.failureCauses = FailureCauses.STATUS_FILE_UPDATION_FAILURE
+        executorErrors.errorMsg = errorMessage
+        // throw new Exception(errorMessage)
+      }
+    }
+
+    def deleteDeltaFunc(index: Int,
+        key: String,
+        iter: Iterator[Row],
+        timestamp: String,
+        rowCountDetailsVO: RowCountDetailsVO
+    ): Iterator[(String, (SegmentUpdateDetails, ExecutionErrors))] = {
+
+      val result = new DeleteDelataResultImpl()
+      var deleteStatus = CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
+      val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+      // here key = segment/blockName
+      val blockName = CarbonUpdateUtil
+        .getBlockName(
+          CarbonTablePath.addDataPartPrefix(key.split(CarbonCommonConstants.FILE_SEPARATOR)(1)))
+      val segmentId = key.split(CarbonCommonConstants.FILE_SEPARATOR)(0)
+      val deleteDeltaBlockDetails: DeleteDeltaBlockDetails = new DeleteDeltaBlockDetails(blockName)
+      val resultIter = new Iterator[(String, (SegmentUpdateDetails, ExecutionErrors))] {
+        val segmentUpdateDetails = new SegmentUpdateDetails()
+        var TID = ""
+        var countOfRows = 0
+        try {
+          while (iter.hasNext) {
+            val oneRow = iter.next
+            TID = oneRow
+              .get(oneRow.fieldIndex(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID)).toString
+            val offset = CarbonUpdateUtil.getRequiredFieldFromTID(TID, TupleIdEnum.OFFSET)
+            val blockletId = CarbonUpdateUtil
+              .getRequiredFieldFromTID(TID, TupleIdEnum.BLOCKLET_ID)
+            val pageId = Integer.parseInt(CarbonUpdateUtil
+              .getRequiredFieldFromTID(TID, TupleIdEnum.PAGE_ID))
+            val IsValidOffset = deleteDeltaBlockDetails.addBlocklet(blockletId, offset, pageId)
+            // stop delete operation
+            if(!IsValidOffset) {
+              executorErrors.failureCauses = FailureCauses.MULTIPLE_INPUT_ROWS_MATCHING
+              executorErrors.errorMsg = "Multiple input rows matched for same row."
+              throw new MultipleMatchingException("Multiple input rows matched for same row.")
+            }
+            countOfRows = countOfRows + 1
+          }
+
+          val blockPath = CarbonUpdateUtil.getTableBlockPath(TID, factPath)
+          val completeBlockName = CarbonTablePath
+            .addDataPartPrefix(CarbonUpdateUtil.getRequiredFieldFromTID(TID, TupleIdEnum.BLOCK_ID) +
+                               CarbonCommonConstants.FACT_FILE_EXT)
+          val deleteDeletaPath = CarbonUpdateUtil
+            .getDeleteDeltaFilePath(blockPath, blockName, timestamp)
+          val carbonDeleteWriter = new CarbonDeleteDeltaWriterImpl(deleteDeletaPath,
+            FileFactory.getFileType(deleteDeletaPath))
+
+
+
+          segmentUpdateDetails.setBlockName(blockName)
+          segmentUpdateDetails.setActualBlockName(completeBlockName)
+          segmentUpdateDetails.setSegmentName(segmentId)
+          segmentUpdateDetails.setDeleteDeltaEndTimestamp(timestamp)
+          segmentUpdateDetails.setDeleteDeltaStartTimestamp(timestamp)
+
+          val alreadyDeletedRows: Long = rowCountDetailsVO.getDeletedRowsInBlock
+          val totalDeletedRows: Long = alreadyDeletedRows + countOfRows
+          segmentUpdateDetails.setDeletedRowsInBlock(totalDeletedRows.toString)
+          if (totalDeletedRows == rowCountDetailsVO.getTotalNumberOfRows) {
+            segmentUpdateDetails.setStatus(CarbonCommonConstants.MARKED_FOR_DELETE)
+          }
+          else {
+            // write the delta file
+            carbonDeleteWriter.write(deleteDeltaBlockDetails)
+          }
+
+          deleteStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
+        } catch {
+          case e : MultipleMatchingException =>
+            LOGGER.audit(e.getMessage)
+            LOGGER.error(e.getMessage)
+          // dont throw exception here.
+          case e: Exception =>
+            val errorMsg = s"Delete data operation is failed for ${ database }.${ tableName }."
+            LOGGER.audit(errorMsg)
+            LOGGER.error(errorMsg + e.getMessage)
+            throw e
+        }
+
+
+        var finished = false
+
+        override def hasNext: Boolean = {
+          if (!finished) {
+            finished = true
+            finished
+          }
+          else {
+            !finished
+          }
+        }
+
+        override def next(): (String, (SegmentUpdateDetails, ExecutionErrors)) = {
+          finished = true
+          result.getKey(deleteStatus, (segmentUpdateDetails, executorErrors))
+        }
+      }
+      resultIter
+    }
+
+    true
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
new file mode 100644
index 0000000..34daf4e
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompaction.scala
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.mutation
+
+import java.util
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.execution.command.AlterTableModel
+import org.apache.spark.sql.execution.command.management.AlterTableCompactionCommand
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager
+import org.apache.carbondata.processing.merger.{CarbonDataMergerUtil, CarbonDataMergerUtilResult, CompactionType}
+
+object HorizontalCompaction {
+
+  val LOG: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+
+  /**
+   * The method does horizontal compaction. After Update and Delete completion
+   * tryHorizontal compaction will be called. In case this method is called after
+   * Update statement then Update Compaction followed by Delete Compaction will be
+   * processed whereas for tryHorizontalCompaction called after Delete statement
+   * then only Delete Compaction will be processed.
+   */
+  def tryHorizontalCompaction(
+      sparkSession: SparkSession,
+      carbonRelation: CarbonRelation,
+      isUpdateOperation: Boolean): Unit = {
+
+    if (!CarbonDataMergerUtil.isHorizontalCompactionEnabled) {
+      return
+    }
+
+    var compactionTypeIUD = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
+    val carbonTable = carbonRelation.tableMeta.carbonTable
+    val absTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+    val updateTimeStamp = System.currentTimeMillis()
+    // To make sure that update and delete timestamps are not same,
+    // required to commit to status metadata and cleanup
+    val deleteTimeStamp = updateTimeStamp + 1
+
+    // get the valid segments
+    var segLists = CarbonDataMergerUtil.getValidSegmentList(absTableIdentifier)
+
+    if (segLists == null || segLists.size() == 0) {
+      return
+    }
+
+    // Should avoid reading Table Status file from Disk every time. Better to load it
+    // in-memory at the starting and pass it along the routines. The constructor of
+    // SegmentUpdateStatusManager reads the Table Status File and Table Update Status
+    // file and save the content in segmentDetails and updateDetails respectively.
+    val segmentUpdateStatusManager: SegmentUpdateStatusManager = new SegmentUpdateStatusManager(
+      absTableIdentifier)
+
+    if (isUpdateOperation) {
+
+      // This is only update operation, perform only update compaction.
+      compactionTypeIUD = CompactionType.IUD_UPDDEL_DELTA_COMPACTION
+      performUpdateDeltaCompaction(sparkSession,
+        compactionTypeIUD,
+        carbonTable,
+        absTableIdentifier,
+        segmentUpdateStatusManager,
+        updateTimeStamp,
+        segLists)
+    }
+
+    // After Update Compaction perform delete compaction
+    compactionTypeIUD = CompactionType.IUD_DELETE_DELTA_COMPACTION
+    segLists = CarbonDataMergerUtil.getValidSegmentList(absTableIdentifier)
+    if (segLists == null || segLists.size() == 0) {
+      return
+    }
+
+    // Delete Compaction
+    performDeleteDeltaCompaction(sparkSession,
+      compactionTypeIUD,
+      carbonTable,
+      absTableIdentifier,
+      segmentUpdateStatusManager,
+      deleteTimeStamp,
+      segLists)
+  }
+
+  /**
+   * Update Delta Horizontal Compaction.
+   */
+  private def performUpdateDeltaCompaction(sparkSession: SparkSession,
+      compactionTypeIUD: CompactionType,
+      carbonTable: CarbonTable,
+      absTableIdentifier: AbsoluteTableIdentifier,
+      segmentUpdateStatusManager: SegmentUpdateStatusManager,
+      factTimeStamp: Long,
+      segLists: util.List[String]): Unit = {
+    val db = carbonTable.getDatabaseName
+    val table = carbonTable.getFactTableName
+    // get the valid segments qualified for update compaction.
+    val validSegList = CarbonDataMergerUtil.getSegListIUDCompactionQualified(segLists,
+      absTableIdentifier,
+      segmentUpdateStatusManager,
+      compactionTypeIUD)
+
+    if (validSegList.size() == 0) {
+      return
+    }
+
+    LOG.info(s"Horizontal Update Compaction operation started for [$db.$table].")
+    LOG.audit(s"Horizontal Update Compaction operation started for [$db.$table].")
+
+    try {
+      // Update Compaction.
+      val alterTableModel = AlterTableModel(Option(carbonTable.getDatabaseName),
+        carbonTable.getFactTableName,
+        Some(segmentUpdateStatusManager),
+        CompactionType.IUD_UPDDEL_DELTA_COMPACTION.toString,
+        Some(factTimeStamp),
+        "")
+
+      AlterTableCompactionCommand(alterTableModel).run(sparkSession)
+    }
+    catch {
+      case e: Exception =>
+        val msg = if (null != e.getMessage) {
+          e.getMessage
+        } else {
+          "Please check logs for more info"
+        }
+        throw new HorizontalCompactionException(
+          s"Horizontal Update Compaction Failed for [${ db }.${ table }]. " + msg, factTimeStamp)
+    }
+    LOG.info(s"Horizontal Update Compaction operation completed for [${ db }.${ table }].")
+    LOG.audit(s"Horizontal Update Compaction operation completed for [${ db }.${ table }].")
+  }
+
+  /**
+   * Delete Delta Horizontal Compaction.
+   */
+  private def performDeleteDeltaCompaction(sparkSession: SparkSession,
+      compactionTypeIUD: CompactionType,
+      carbonTable: CarbonTable,
+      absTableIdentifier: AbsoluteTableIdentifier,
+      segmentUpdateStatusManager: SegmentUpdateStatusManager,
+      factTimeStamp: Long,
+      segLists: util.List[String]): Unit = {
+
+    val db = carbonTable.getDatabaseName
+    val table = carbonTable.getFactTableName
+    val deletedBlocksList = CarbonDataMergerUtil.getSegListIUDCompactionQualified(segLists,
+      absTableIdentifier,
+      segmentUpdateStatusManager,
+      compactionTypeIUD)
+
+    if (deletedBlocksList.size() == 0) {
+      return
+    }
+
+    LOG.info(s"Horizontal Delete Compaction operation started for [$db.$table].")
+    LOG.audit(s"Horizontal Delete Compaction operation started for [$db.$table].")
+
+    try {
+
+      // Delete Compaction RDD
+      val rdd1 = sparkSession.sparkContext
+        .parallelize(deletedBlocksList.asScala, deletedBlocksList.size())
+
+      val timestamp = factTimeStamp
+      val updateStatusDetails = segmentUpdateStatusManager.getUpdateStatusDetails
+      val result = rdd1.mapPartitions(iter =>
+        new Iterator[Seq[CarbonDataMergerUtilResult]] {
+          override def hasNext: Boolean = iter.hasNext
+
+          override def next(): Seq[CarbonDataMergerUtilResult] = {
+            val segmentAndBlocks = iter.next
+            val segment = segmentAndBlocks.substring(0, segmentAndBlocks.lastIndexOf("/"))
+            val blockName = segmentAndBlocks
+              .substring(segmentAndBlocks.lastIndexOf("/") + 1, segmentAndBlocks.length)
+
+            val result = CarbonDataMergerUtil.compactBlockDeleteDeltaFiles(segment, blockName,
+              absTableIdentifier,
+              updateStatusDetails,
+              timestamp)
+
+            result.asScala.toList
+
+          }
+        }).collect
+
+      val resultList = ListBuffer[CarbonDataMergerUtilResult]()
+      result.foreach(x => {
+        x.foreach(y => {
+          resultList += y
+        })
+      })
+
+      val updateStatus = CarbonDataMergerUtil.updateStatusFile(resultList.toList.asJava,
+        carbonTable,
+        timestamp.toString,
+        segmentUpdateStatusManager)
+      if (updateStatus == false) {
+        LOG.audit(s"Delete Compaction data operation is failed for [$db.$table].")
+        LOG.error("Delete Compaction data operation is failed.")
+        throw new HorizontalCompactionException(
+          s"Horizontal Delete Compaction Failed for [$db.$table] ." +
+          s" Please check logs for more info.", factTimeStamp)
+      }
+      else {
+        LOG.info(s"Horizontal Delete Compaction operation completed for [$db.$table].")
+        LOG.audit(s"Horizontal Delete Compaction operation completed for [$db.$table].")
+      }
+    }
+    catch {
+      case e: Exception =>
+        val msg = if (null != e.getMessage) {
+          e.getMessage
+        } else {
+          "Please check logs for more info"
+        }
+        throw new HorizontalCompactionException(
+          s"Horizontal Delete Compaction Failed for [${ db }.${ table }]. " + msg, factTimeStamp)
+    }
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompactionException.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompactionException.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompactionException.scala
new file mode 100644
index 0000000..60ae078
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/HorizontalCompactionException.scala
@@ -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.spark.sql.execution.command.mutation
+
+class HorizontalCompactionException(
+    message: String,
+    // required for cleanup
+    val compactionTimeStamp: Long) extends RuntimeException(message) {
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
new file mode 100644
index 0000000..9cf8a91
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForDeleteCommand.scala
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.mutation
+
+import org.apache.spark.sql.{CarbonEnv, Dataset, Row, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.hive.CarbonRelation
+
+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.spark.load.FailureCauses
+
+/**
+ * IUD update delete and compaction framework.
+ *
+ */
+private[sql] case class ProjectForDeleteCommand(
+    plan: LogicalPlan,
+    identifier: Seq[String],
+    timestamp: String) extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+    val dataFrame = Dataset.ofRows(sparkSession, plan)
+    //    dataFrame.show(truncate = false)
+    //    dataFrame.collect().foreach(println)
+    val dataRdd = dataFrame.rdd
+
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(DeleteExecution.getTableIdentifier(identifier))(sparkSession).
+      asInstanceOf[CarbonRelation]
+    val carbonTable = relation.tableMeta.carbonTable
+    val metadataLock = CarbonLockFactory
+      .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+        LockUsage.METADATA_LOCK)
+    var lockStatus = false
+    try {
+      lockStatus = metadataLock.lockWithRetries()
+      LOGGER.audit(s" Delete data request has been received " +
+                   s"for ${ relation.databaseName }.${ relation.tableName }.")
+      if (lockStatus) {
+        LOGGER.info("Successfully able to get the table metadata file lock")
+      } else {
+        throw new Exception("Table is locked for deletion. Please try after some time")
+      }
+      val executorErrors = ExecutionErrors(FailureCauses.NONE, "")
+
+      // handle the clean up of IUD.
+      CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, false)
+
+      if (DeleteExecution
+        .deleteDeltaExecution(identifier, sparkSession, dataRdd, timestamp, relation,
+          isUpdateOperation = false, executorErrors)) {
+        // call IUD Compaction.
+        HorizontalCompaction.tryHorizontalCompaction(sparkSession, relation,
+          isUpdateOperation = false)
+      }
+    } catch {
+      case e: HorizontalCompactionException =>
+        LOGGER.error("Delete operation passed. Exception in Horizontal Compaction." +
+                     " Please check logs. " + e.getMessage)
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, e.compactionTimeStamp.toString)
+
+      case e: Exception =>
+        LOGGER.error(e, "Exception in Delete data operation " + e.getMessage)
+        // ****** start clean up.
+        // In case of failure , clean all related delete delta files
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, timestamp)
+
+        // clean up. Null check is required as for executor error some times message is null
+        if (null != e.getMessage) {
+          sys.error("Delete data operation is failed. " + e.getMessage)
+        }
+        else {
+          sys.error("Delete data operation is failed. Please check logs.")
+        }
+    } finally {
+      if (lockStatus) {
+        CarbonLockUtil.fileUnlock(metadataLock, LockUsage.METADATA_LOCK)
+      }
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
new file mode 100644
index 0000000..036ca49
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/ProjectForUpdateCommand.scala
@@ -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.spark.sql.execution.command.mutation
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.management.LoadTableCommand
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.storage.StorageLevel
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.{CarbonLockFactory, CarbonLockUtil, LockUsage}
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.spark.load.FailureCauses
+
+private[sql] case class ProjectForUpdateCommand(
+    plan: LogicalPlan, tableIdentifier: Seq[String])
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER = LogServiceFactory.getLogService(ProjectForUpdateCommand.getClass.getName)
+
+    //  sqlContext.sparkContext.setLocalProperty(org.apache.spark.sql.execution.SQLExecution
+    //  .EXECUTION_ID_KEY, null)
+    // DataFrame(sqlContext, plan).show(truncate = false)
+    // return Seq.empty
+
+
+    val res = plan find {
+      case relation: LogicalRelation if relation.relation
+        .isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        true
+      case _ => false
+    }
+
+    if (res.isEmpty) {
+      return Seq.empty
+    }
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(DeleteExecution.getTableIdentifier(tableIdentifier))(sparkSession).
+      asInstanceOf[CarbonRelation]
+    //    val relation = CarbonEnv.get.carbonMetastore
+    //      .lookupRelation1(deleteExecution.getTableIdentifier(tableIdentifier))(sqlContext).
+    //      asInstanceOf[CarbonRelation]
+    val carbonTable = relation.tableMeta.carbonTable
+    val metadataLock = CarbonLockFactory
+      .getCarbonLockObj(carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier,
+        LockUsage.METADATA_LOCK)
+    var lockStatus = false
+    // get the current time stamp which should be same for delete and update.
+    val currentTime = CarbonUpdateUtil.readCurrentTime
+    //    var dataFrame: DataFrame = null
+    var dataSet: DataFrame = null
+    var isPersistEnabled = CarbonProperties.getInstance.isPersistUpdateDataset()
+    try {
+      lockStatus = metadataLock.lockWithRetries()
+      if (lockStatus) {
+        logInfo("Successfully able to get the table metadata file lock")
+      }
+      else {
+        throw new Exception("Table is locked for updation. Please try after some time")
+      }
+      val tablePath = CarbonStorePath.getCarbonTablePath(
+        carbonTable.getStorePath,
+        carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier)
+      // Get RDD.
+
+      dataSet = if (isPersistEnabled) {
+        Dataset.ofRows(sparkSession, plan).persist(StorageLevel.fromString(
+          CarbonProperties.getInstance.getUpdateDatasetStorageLevel()))
+      }
+      else {
+        Dataset.ofRows(sparkSession, plan)
+      }
+      var executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
+
+
+      // handle the clean up of IUD.
+      CarbonUpdateUtil.cleanUpDeltaFiles(carbonTable, false)
+
+      // do delete operation.
+      DeleteExecution.deleteDeltaExecution(tableIdentifier, sparkSession, dataSet.rdd,
+        currentTime + "",
+        relation, isUpdateOperation = true, executionErrors)
+
+      if(executionErrors.failureCauses != FailureCauses.NONE) {
+        throw new Exception(executionErrors.errorMsg)
+      }
+
+      // do update operation.
+      performUpdate(dataSet, tableIdentifier, plan, sparkSession, currentTime, executionErrors)
+
+      if(executionErrors.failureCauses != FailureCauses.NONE) {
+        throw new Exception(executionErrors.errorMsg)
+      }
+
+      // Do IUD Compaction.
+      HorizontalCompaction.tryHorizontalCompaction(sparkSession, relation, isUpdateOperation = true)
+    } catch {
+      case e: HorizontalCompactionException =>
+        LOGGER.error(
+          "Update operation passed. Exception in Horizontal Compaction. Please check logs." + e)
+        // In case of failure , clean all related delta files
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, e.compactionTimeStamp.toString)
+
+      case e: Exception =>
+        LOGGER.error("Exception in update operation" + e)
+        // ****** start clean up.
+        // In case of failure , clean all related delete delta files
+        CarbonUpdateUtil.cleanStaleDeltaFiles(carbonTable, currentTime + "")
+
+        // *****end clean up.
+        if (null != e.getMessage) {
+          sys.error("Update operation failed. " + e.getMessage)
+        }
+        if (null != e.getCause && null != e.getCause.getMessage) {
+          sys.error("Update operation failed. " + e.getCause.getMessage)
+        }
+        sys.error("Update operation failed. please check logs.")
+    }
+    finally {
+      if (null != dataSet && isPersistEnabled) {
+        dataSet.unpersist()
+      }
+      if (lockStatus) {
+        CarbonLockUtil.fileUnlock(metadataLock, LockUsage.METADATA_LOCK)
+      }
+    }
+    Seq.empty
+  }
+
+  private def performUpdate(
+      dataFrame: Dataset[Row],
+      tableIdentifier: Seq[String],
+      plan: LogicalPlan,
+      sparkSession: SparkSession,
+      currentTime: Long,
+      executorErrors: ExecutionErrors): Unit = {
+
+    def isDestinationRelation(relation: CarbonDatasourceHadoopRelation): Boolean = {
+
+      val tableName = relation.identifier.getCarbonTableIdentifier.getTableName
+      val dbName = relation.identifier.getCarbonTableIdentifier.getDatabaseName
+      (tableIdentifier.size > 1 &&
+       tableIdentifier(0) == dbName &&
+       tableIdentifier(1) == tableName) ||
+      (tableIdentifier(0) == tableName)
+    }
+    def getHeader(relation: CarbonDatasourceHadoopRelation, plan: LogicalPlan): String = {
+      var header = ""
+      var found = false
+
+      plan match {
+        case Project(pList, _) if (!found) =>
+          found = true
+          header = pList
+            .filter(field => !field.name
+              .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
+            .map(col => if (col.name.endsWith(CarbonCommonConstants.UPDATED_COL_EXTENSION)) {
+              col.name
+                .substring(0, col.name.lastIndexOf(CarbonCommonConstants.UPDATED_COL_EXTENSION))
+            }
+            else {
+              col.name
+            }).mkString(",")
+      }
+      header
+    }
+    val ex = dataFrame.queryExecution.analyzed
+    val res = ex find {
+      case relation: LogicalRelation
+        if relation.relation.isInstanceOf[CarbonDatasourceHadoopRelation] &&
+           isDestinationRelation(relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]) =>
+        true
+      case _ => false
+    }
+    val carbonRelation: CarbonDatasourceHadoopRelation = res match {
+      case Some(relation: LogicalRelation) =>
+        relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+      case _ => sys.error("")
+    }
+
+    val updateTableModel = UpdateTableModel(true, currentTime, executorErrors)
+
+    val header = getHeader(carbonRelation, plan)
+
+    LoadTableCommand(
+      Some(carbonRelation.identifier.getCarbonTableIdentifier.getDatabaseName),
+      carbonRelation.identifier.getCarbonTableIdentifier.getTableName,
+      null,
+      Seq(),
+      Map(("fileheader" -> header)),
+      false,
+      null,
+      Some(dataFrame),
+      Some(updateTableModel)).run(sparkSession)
+
+    executorErrors.errorMsg = updateTableModel.executorErrors.errorMsg
+    executorErrors.failureCauses = updateTableModel.executorErrors.failureCauses
+
+    Seq.empty
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/package.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/package.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/package.scala
new file mode 100644
index 0000000..07ef555
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/package.scala
@@ -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.spark.sql.execution.command
+
+import scala.language.implicitConversions
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+object Checker {
+  def validateTableExists(
+      dbName: Option[String],
+      tableName: String,
+      session: SparkSession): Unit = {
+    val identifier = TableIdentifier(tableName, dbName)
+    if (!CarbonEnv.getInstance(session).carbonMetastore.tableExists(identifier)(session)) {
+      val err = s"table $dbName.$tableName not found"
+      LogServiceFactory.getLogService(this.getClass.getName).error(err)
+      throw new IllegalArgumentException(err)
+    }
+  }
+}
+
+/**
+ * Interface for command that modifies schema
+ */
+trait SchemaProcessCommand {
+  def processSchema(sparkSession: SparkSession): Seq[Row]
+}
+
+/**
+ * Interface for command that need to process data in file system
+ */
+trait DataProcessCommand {
+  def processData(sparkSession: SparkSession): Seq[Row]
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
new file mode 100644
index 0000000..1f06aed
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableDropCarbonPartitionCommand.scala
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.partition
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableDropPartitionModel, DataProcessCommand, RunnableCommand, SchemaProcessCommand}
+import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
+import org.apache.spark.util.AlterTableUtil
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.cache.CacheProvider
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.PartitionInfo
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+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.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+
+case class AlterTableDropCarbonPartitionCommand(
+    model: AlterTableDropPartitionModel)
+  extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
+  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    if (model.partitionId.equals("0")) {
+      sys.error(s"Cannot drop default partition! Please use delete statement!")
+    }
+    processSchema(sparkSession)
+    processData(sparkSession)
+    Seq.empty
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+    val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
+    val tableName = model.tableName
+    val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
+      .asInstanceOf[CarbonRelation]
+    val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+    val storePath = relation.tableMeta.storePath
+    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
+    if (relation == null) {
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
+      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
+      sys.error(s"Alter table failed. table not found: $dbName.$tableName")
+    }
+    val table = relation.tableMeta.carbonTable
+    val partitionInfo = table.getPartitionInfo(tableName)
+    if (partitionInfo == null) {
+      sys.error(s"Table $tableName is not a partition table.")
+    }
+    val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
+    // keep a copy of partitionIdList before update partitionInfo.
+    // will be used in partition data scan
+    oldPartitionIds.addAll(partitionIds.asJava)
+    val partitionIndex = partitionIds.indexOf(Integer.valueOf(model.partitionId))
+    partitionInfo.getPartitionType match {
+      case PartitionType.HASH => sys.error(s"Hash partition cannot be dropped!")
+      case PartitionType.RANGE =>
+        val rangeInfo = new util.ArrayList(partitionInfo.getRangeInfo)
+        val rangeToRemove = partitionInfo.getRangeInfo.get(partitionIndex - 1)
+        rangeInfo.remove(rangeToRemove)
+        partitionInfo.setRangeInfo(rangeInfo)
+      case PartitionType.LIST =>
+        val listInfo = new util.ArrayList(partitionInfo.getListInfo)
+        val listToRemove = partitionInfo.getListInfo.get(partitionIndex - 1)
+        listInfo.remove(listToRemove)
+        partitionInfo.setListInfo(listInfo)
+      case PartitionType.RANGE_INTERVAL =>
+        sys.error(s"Dropping range interval partition isn't support yet!")
+    }
+    partitionInfo.dropPartition(partitionIndex)
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
+    val schemaFilePath = carbonTablePath.getSchemaFilePath
+    // read TableInfo
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
+
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
+    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
+      dbName, tableName, storePath)
+    val tableSchema = wrapperTableInfo.getFactTable
+    tableSchema.setPartitionInfo(partitionInfo)
+    wrapperTableInfo.setFactTable(tableSchema)
+    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
+    val thriftTable =
+      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+    thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
+      .setTime_stamp(System.currentTimeMillis)
+    carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
+      dbName, tableName, storePath)
+    CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
+    // update the schema modified time
+    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
+    // sparkSession.catalog.refreshTable(tableName)
+    Seq.empty
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+    val dbName = model.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
+    val tableName = model.tableName
+    var locks = List.empty[ICarbonLock]
+    var success = false
+    try {
+      val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+        LockUsage.COMPACTION_LOCK,
+        LockUsage.DELETE_SEGMENT_LOCK,
+        LockUsage.DROP_TABLE_LOCK,
+        LockUsage.CLEAN_FILES_LOCK,
+        LockUsage.ALTER_PARTITION_LOCK)
+      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
+        locksToBeAcquired)(sparkSession)
+      val carbonLoadModel = new CarbonLoadModel()
+      val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+      val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+      val table = relation.tableMeta.carbonTable
+      val dataLoadSchema = new CarbonDataLoadSchema(table)
+      // Need to fill dimension relation
+      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+      carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
+      carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
+      carbonLoadModel.setStorePath(relation.tableMeta.storePath)
+      val loadStartTime = CarbonUpdateUtil.readCurrentTime
+      carbonLoadModel.setFactTimeStamp(loadStartTime)
+      CarbonDataRDDFactory.alterTableDropPartition(
+        sparkSession.sqlContext,
+        model.partitionId,
+        carbonLoadModel,
+        model.dropWithData,
+        oldPartitionIds.asScala.toList
+      )
+      success = true
+    } catch {
+      case e: Exception =>
+        sys.error(s"Drop Partition failed. Please check logs for more info. ${ e.getMessage } ")
+        success = false
+    } finally {
+      CacheProvider.getInstance().dropAllCache()
+      AlterTableUtil.releaseLocks(locks)
+      LOGGER.info("Locks released after alter table drop partition action.")
+      LOGGER.audit("Locks released after alter table drop partition action.")
+    }
+    LOGGER.info(s"Alter table drop partition is successful for table $dbName.$tableName")
+    LOGGER.audit(s"Alter table drop partition is successful for table $dbName.$tableName")
+    Seq.empty
+  }
+}


[06/50] [abbrv] carbondata git commit: [CARBONDATA-1515]Fixed NPE in data loading in long run

Posted by ja...@apache.org.
[CARBONDATA-1515]Fixed NPE in data loading in long run

Scenario:
Data size: 3.5 billion rows(4.1 tb data)
3 node cluster
Number of core while data loading 12.
No. of loads 100 times
Problem: In DataConverterProcessorStepImpl it is using array list for adding all the local converter, in case of multiple thread scenario it is creating a hole (null value)(as array list if not synchronized). while closing the converter it is it is throwing NPE
Solution: Add local converter in synchronized block

This closes #1384


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

Branch: refs/heads/streaming_ingest
Commit: 2ec69f6179832a394886f26b921f1dc3e916abe8
Parents: 2cd22e1
Author: kumarvishal <ku...@gmail.com>
Authored: Mon Sep 25 18:27:03 2017 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Tue Sep 26 20:16:51 2017 +0530

----------------------------------------------------------------------
 .../newflow/steps/DataConverterProcessorStepImpl.java        | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/2ec69f61/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java
index de96cc9..728aa18 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java
@@ -87,7 +87,9 @@ public class DataConverterProcessorStepImpl extends AbstractDataLoadProcessorSte
         if (first) {
           first = false;
           localConverter = converters.get(0).createCopyForNewThread();
-          converters.add(localConverter);
+          synchronized (converters) {
+            converters.add(localConverter);
+          }
         }
         return childIter.hasNext();
       }
@@ -185,7 +187,9 @@ public class DataConverterProcessorStepImpl extends AbstractDataLoadProcessorSte
       super.close();
       if (converters != null) {
         for (RowConverter converter : converters) {
-          converter.finish();
+          if (null != converter) {
+            converter.finish();
+          }
         }
       }
     }


[24/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkHolder.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkHolder.java
deleted file mode 100644
index 10b3ad5..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkHolder.java
+++ /dev/null
@@ -1,522 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.sortandgroupby.sortdata;
-
-import java.io.BufferedInputStream;
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-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.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
-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.NonDictionaryUtil;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHolder> {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SortTempFileChunkHolder.class.getName());
-
-  /**
-   * temp file
-   */
-  private File tempFile;
-
-  /**
-   * read stream
-   */
-  private DataInputStream stream;
-
-  /**
-   * entry count
-   */
-  private int entryCount;
-
-  /**
-   * number record read
-   */
-  private int numberOfObjectRead;
-
-  /**
-   * return row
-   */
-  private Object[] returnRow;
-
-  /**
-   * number of measures
-   */
-  private int measureCount;
-
-  /**
-   * number of dimensionCount
-   */
-  private int dimensionCount;
-
-  /**
-   * number of complexDimensionCount
-   */
-  private int complexDimensionCount;
-
-  /**
-   * fileBufferSize for file reader stream size
-   */
-  private int fileBufferSize;
-
-  private Object[][] currentBuffer;
-
-  private Object[][] backupBuffer;
-
-  private boolean isBackupFilled;
-
-  private boolean prefetch;
-
-  private int bufferSize;
-
-  private int bufferRowCounter;
-
-  private ExecutorService executorService;
-
-  private Future<Void> submit;
-
-  private int prefetchRecordsProceesed;
-
-  /**
-   * sortTempFileNoOFRecordsInCompression
-   */
-  private int sortTempFileNoOFRecordsInCompression;
-
-  /**
-   * isSortTempFileCompressionEnabled
-   */
-  private boolean isSortTempFileCompressionEnabled;
-
-  /**
-   * totalRecordFetch
-   */
-  private int totalRecordFetch;
-
-  private int noDictionaryCount;
-
-  private DataType[] aggType;
-
-  /**
-   * to store whether dimension is of dictionary type or not
-   */
-  private boolean[] isNoDictionaryDimensionColumn;
-
-  /**
-   * to store whether sort column is of dictionary type or not
-   */
-  private boolean[] isNoDictionarySortColumn;
-
-  /**
-   * Constructor to initialize
-   *
-   * @param tempFile
-   * @param dimensionCount
-   * @param complexDimensionCount
-   * @param measureCount
-   * @param fileBufferSize
-   * @param noDictionaryCount
-   * @param aggType
-   * @param isNoDictionaryDimensionColumn
-   */
-  public SortTempFileChunkHolder(File tempFile, int dimensionCount, int complexDimensionCount,
-      int measureCount, int fileBufferSize, int noDictionaryCount, DataType[] aggType,
-      boolean[] isNoDictionaryDimensionColumn, boolean[] isNoDictionarySortColumn) {
-    // set temp file
-    this.tempFile = tempFile;
-
-    // set measure and dimension count
-    this.measureCount = measureCount;
-    this.dimensionCount = dimensionCount;
-    this.complexDimensionCount = complexDimensionCount;
-
-    this.noDictionaryCount = noDictionaryCount;
-    // set mdkey length
-    this.fileBufferSize = fileBufferSize;
-    this.executorService = Executors.newFixedThreadPool(1);
-    this.aggType = aggType;
-
-    this.isNoDictionaryDimensionColumn = isNoDictionaryDimensionColumn;
-    this.isNoDictionarySortColumn = isNoDictionarySortColumn;
-  }
-
-  /**
-   * This method will be used to initialize
-   *
-   * @throws CarbonSortKeyAndGroupByException problem while initializing
-   */
-  public void initialize() throws CarbonSortKeyAndGroupByException {
-    prefetch = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_PREFETCH,
-            CarbonCommonConstants.CARBON_MERGE_SORT_PREFETCH_DEFAULT));
-    bufferSize = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
-            CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT));
-    this.isSortTempFileCompressionEnabled = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED,
-            CarbonCommonConstants.IS_SORT_TEMP_FILE_COMPRESSION_ENABLED_DEFAULTVALUE));
-    if (this.isSortTempFileCompressionEnabled) {
-      LOGGER.info("Compression was used while writing the sortTempFile");
-    }
-
-    try {
-      this.sortTempFileNoOFRecordsInCompression = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION,
-              CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE));
-      if (this.sortTempFileNoOFRecordsInCompression < 1) {
-        LOGGER.error("Invalid value for: "
-            + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
-            + ": Only Positive Integer value(greater than zero) is allowed.Default value will"
-            + " be used");
-
-        this.sortTempFileNoOFRecordsInCompression = Integer.parseInt(
-            CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.error(
-          "Invalid value for: " + CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORDS_FOR_COMPRESSION
-              + ", only Positive Integer value is allowed.Default value will be used");
-      this.sortTempFileNoOFRecordsInCompression = Integer
-          .parseInt(CarbonCommonConstants.SORT_TEMP_FILE_NO_OF_RECORD_FOR_COMPRESSION_DEFAULTVALUE);
-    }
-
-    initialise();
-  }
-
-  private void initialise() throws CarbonSortKeyAndGroupByException {
-    try {
-      if (isSortTempFileCompressionEnabled) {
-        this.bufferSize = sortTempFileNoOFRecordsInCompression;
-      }
-      stream = new DataInputStream(
-          new BufferedInputStream(new FileInputStream(tempFile), this.fileBufferSize));
-      this.entryCount = stream.readInt();
-      if (prefetch) {
-        new DataFetcher(false).call();
-        totalRecordFetch += currentBuffer.length;
-        if (totalRecordFetch < this.entryCount) {
-          submit = executorService.submit(new DataFetcher(true));
-        }
-      } else {
-        if (isSortTempFileCompressionEnabled) {
-          new DataFetcher(false).call();
-        }
-      }
-
-    } catch (FileNotFoundException e) {
-      LOGGER.error(e);
-      throw new CarbonSortKeyAndGroupByException(tempFile + " No Found", e);
-    } catch (IOException e) {
-      LOGGER.error(e);
-      throw new CarbonSortKeyAndGroupByException(tempFile + " No Found", e);
-    } catch (Exception e) {
-      LOGGER.error(e);
-      throw new CarbonSortKeyAndGroupByException(tempFile + " Problem while reading", e);
-    }
-  }
-
-  /**
-   * This method will be used to read new row from file
-   *
-   * @throws CarbonSortKeyAndGroupByException problem while reading
-   */
-  public void readRow() throws CarbonSortKeyAndGroupByException {
-    if (prefetch) {
-      fillDataForPrefetch();
-    } else if (isSortTempFileCompressionEnabled) {
-      if (bufferRowCounter >= bufferSize) {
-        try {
-          new DataFetcher(false).call();
-          bufferRowCounter = 0;
-        } catch (Exception e) {
-          LOGGER.error(e);
-          throw new CarbonSortKeyAndGroupByException(tempFile + " Problem while reading", e);
-        }
-
-      }
-      prefetchRecordsProceesed++;
-      returnRow = currentBuffer[bufferRowCounter++];
-    } else {
-      this.returnRow = getRowFromStream();
-    }
-  }
-
-  private void fillDataForPrefetch() {
-    if (bufferRowCounter >= bufferSize) {
-      if (isBackupFilled) {
-        bufferRowCounter = 0;
-        currentBuffer = backupBuffer;
-        totalRecordFetch += currentBuffer.length;
-        isBackupFilled = false;
-        if (totalRecordFetch < this.entryCount) {
-          submit = executorService.submit(new DataFetcher(true));
-        }
-      } else {
-        try {
-          submit.get();
-        } catch (Exception e) {
-          LOGGER.error(e);
-        }
-        bufferRowCounter = 0;
-        currentBuffer = backupBuffer;
-        isBackupFilled = false;
-        totalRecordFetch += currentBuffer.length;
-        if (totalRecordFetch < this.entryCount) {
-          submit = executorService.submit(new DataFetcher(true));
-        }
-      }
-    }
-    prefetchRecordsProceesed++;
-    returnRow = currentBuffer[bufferRowCounter++];
-  }
-
-  /**
-   * Reads row from file
-   * @return Object[]
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  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[this.dimensionCount - this.noDictionaryCount];
-    byte[][] nonDicArray = new byte[this.noDictionaryCount + this.complexDimensionCount][];
-    Object[] measures = new Object[this.measureCount];
-    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 < complexDimensionCount; 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 < this.measureCount; i++) {
-        if (stream.readByte() == 1) {
-          switch (aggType[i]) {
-            case SHORT:
-              measures[index++] = stream.readShort();
-              break;
-            case INT:
-              measures[index++] = stream.readInt();
-              break;
-            case LONG:
-              measures[index++] = stream.readLong();
-              break;
-            case DOUBLE:
-              measures[index++] = stream.readDouble();
-              break;
-            case DECIMAL:
-              int len = stream.readInt();
-              byte[] buff = new byte[len];
-              stream.readFully(buff);
-              measures[index++] = DataTypeUtil.byteToBigDecimal(buff);
-              break;
-            default:
-              throw new IllegalArgumentException("unsupported data type:" + aggType[i]);
-          }
-        } 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);
-    }
-
-    //return out row
-    return holder;
-  }
-
-  /**
-   * below method will be used to get the row
-   *
-   * @return row
-   */
-  public Object[] getRow() {
-    return this.returnRow;
-  }
-
-  /**
-   * below method will be used to check whether any more records are present
-   * in file or not
-   *
-   * @return more row present in file
-   */
-  public boolean hasNext() {
-    if (prefetch || isSortTempFileCompressionEnabled) {
-      return this.prefetchRecordsProceesed < this.entryCount;
-    }
-    return this.numberOfObjectRead < this.entryCount;
-  }
-
-  /**
-   * Below method will be used to close streams
-   */
-  public void closeStream() {
-    CarbonUtil.closeStreams(stream);
-    executorService.shutdown();
-    this.backupBuffer = null;
-    this.currentBuffer = null;
-  }
-
-  /**
-   * This method will number of entries
-   *
-   * @return entryCount
-   */
-  public int getEntryCount() {
-    return entryCount;
-  }
-
-  @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;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-
-    if (!(obj instanceof SortTempFileChunkHolder)) {
-      return false;
-    }
-    SortTempFileChunkHolder o = (SortTempFileChunkHolder) obj;
-
-    return this == o;
-  }
-
-  @Override public int hashCode() {
-    int hash = 0;
-    hash += 31 * measureCount;
-    hash += 31 * dimensionCount;
-    hash += 31 * complexDimensionCount;
-    hash += 31 * noDictionaryCount;
-    hash += tempFile.hashCode();
-    return hash;
-  }
-
-  private final class DataFetcher implements Callable<Void> {
-    private boolean isBackUpFilling;
-
-    private int numberOfRecords;
-
-    private DataFetcher(boolean backUp) {
-      isBackUpFilling = backUp;
-      calculateNumberOfRecordsToBeFetched();
-    }
-
-    private void calculateNumberOfRecordsToBeFetched() {
-      int numberOfRecordsLeftToBeRead = entryCount - totalRecordFetch;
-      numberOfRecords =
-          bufferSize < numberOfRecordsLeftToBeRead ? bufferSize : numberOfRecordsLeftToBeRead;
-    }
-
-    @Override public Void call() throws Exception {
-      try {
-        if (isBackUpFilling) {
-          backupBuffer = prefetchRecordsFromFile(numberOfRecords);
-          isBackupFilled = true;
-        } else {
-          currentBuffer = prefetchRecordsFromFile(numberOfRecords);
-        }
-      } catch (Exception e) {
-        LOGGER.error(e);
-      }
-      return null;
-    }
-
-  }
-
-  /**
-   * This method will read the records from sort temp file and keep it in a buffer
-   *
-   * @param numberOfRecords
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkWriter.java
deleted file mode 100644
index f0bac85..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/SortTempFileChunkWriter.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sortandgroupby.sortdata;
-
-import java.io.File;
-
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-public class SortTempFileChunkWriter implements TempSortFileWriter {
-  /**
-   * writer
-   */
-  private TempSortFileWriter writer;
-
-  /**
-   * recordPerLeaf
-   */
-  private int recordPerLeaf;
-
-  /**
-   * CarbonCompressedSortTempFileChunkWriter
-   *
-   * @param writer
-   */
-  public SortTempFileChunkWriter(TempSortFileWriter writer, int recordPerLeaf) {
-    this.writer = writer;
-    this.recordPerLeaf = recordPerLeaf;
-  }
-
-  /**
-   * initialize
-   */
-  public void initiaize(File file, int entryCount) throws CarbonSortKeyAndGroupByException {
-    this.writer.initiaize(file, entryCount);
-  }
-
-  /**
-   * finish
-   */
-  public void finish() {
-    this.writer.finish();
-  }
-
-  /**
-   * Below method will be used to write the sort temp file chunk by chunk
-   */
-  public void writeSortTempFile(Object[][] records) throws CarbonSortKeyAndGroupByException {
-    int recordCount = 0;
-    Object[][] tempRecords;
-    while (recordCount < records.length) {
-      if (records.length - recordCount < recordPerLeaf) {
-        recordPerLeaf = records.length - recordCount;
-      }
-      tempRecords = new Object[recordPerLeaf][];
-      System.arraycopy(records, recordCount, tempRecords, 0, recordPerLeaf);
-      recordCount += recordPerLeaf;
-      this.writer.writeSortTempFile(tempRecords);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileReader.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileReader.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileReader.java
deleted file mode 100644
index 2bf657e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileReader.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sortandgroupby.sortdata;
-
-public interface TempSortFileReader {
-  /**
-   * below method will be used to close the file holder
-   */
-  void finish();
-
-  /**
-   * Below method will be used to get the row
-   */
-  Object[][] getRow();
-
-  /**
-   * Below method will be used to get the total row count in temp file
-   *
-   * @return
-   */
-  int getEntryCount();
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileWriter.java
deleted file mode 100644
index 6679c8e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileWriter.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sortandgroupby.sortdata;
-
-import java.io.File;
-
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-public interface TempSortFileWriter {
-  /**
-   * Method will be used to initialize
-   *
-   * @param file
-   * @param entryCount
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  void initiaize(File file, int entryCount) throws CarbonSortKeyAndGroupByException;
-
-  /**
-   * Method will be used to finish
-   */
-  void finish();
-
-  /**
-   * Below method will be used to write the sort temp file
-   *
-   * @param records
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  void writeSortTempFile(Object[][] records) throws CarbonSortKeyAndGroupByException;
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileWriterFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileWriterFactory.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileWriterFactory.java
deleted file mode 100644
index c0e8c6e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/TempSortFileWriterFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sortandgroupby.sortdata;
-
-public final class TempSortFileWriterFactory {
-  private static final TempSortFileWriterFactory WRITERFACTORY = new TempSortFileWriterFactory();
-
-  private TempSortFileWriterFactory() {
-
-  }
-
-  public static TempSortFileWriterFactory getInstance() {
-    return WRITERFACTORY;
-  }
-
-  public TempSortFileWriter getTempSortFileWriter(boolean isCompressionEnabled, int dimensionCount,
-      int complexDimensionCount, int measureCount, int noDictionaryCount, int writeBufferSize) {
-    if (isCompressionEnabled) {
-      return new CompressedTempSortFileWriter(dimensionCount, complexDimensionCount, measureCount,
-          noDictionaryCount, writeBufferSize);
-    } else {
-      return new UnCompressedTempSortFileWriter(dimensionCount, complexDimensionCount, measureCount,
-          noDictionaryCount, writeBufferSize);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/UnCompressedTempSortFileWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/UnCompressedTempSortFileWriter.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/UnCompressedTempSortFileWriter.java
deleted file mode 100644
index 51b3964..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdata/UnCompressedTempSortFileWriter.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.sortandgroupby.sortdata;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-
-public class UnCompressedTempSortFileWriter extends AbstractTempSortFileWriter {
-
-  /**
-   * UnCompressedTempSortFileWriter
-   *
-   * @param writeBufferSize
-   * @param dimensionCount
-   * @param measureCount
-   */
-  public UnCompressedTempSortFileWriter(int dimensionCount, int complexDimensionCount,
-      int measureCount, int noDictionaryCount, int writeBufferSize) {
-    super(dimensionCount, complexDimensionCount, measureCount, noDictionaryCount, writeBufferSize);
-  }
-
-  public static void writeDataOutputStream(Object[][] records, DataOutputStream dataOutputStream,
-      int measureCount, int dimensionCount, int noDictionaryCount, int complexDimensionCount)
-      throws IOException {
-    Object[] row;
-    for (int recordIndex = 0; recordIndex < records.length; recordIndex++) {
-      row = records[recordIndex];
-      int fieldIndex = 0;
-
-      for (int counter = 0; counter < dimensionCount; counter++) {
-        dataOutputStream.writeInt((Integer) NonDictionaryUtil.getDimension(fieldIndex++, row));
-      }
-
-      //write byte[] of high card dims
-      if (noDictionaryCount > 0) {
-        dataOutputStream.write(NonDictionaryUtil.getByteArrayForNoDictionaryCols(row));
-      }
-      fieldIndex = 0;
-      for (int counter = 0; counter < complexDimensionCount; counter++) {
-        int complexByteArrayLength = ((byte[]) row[fieldIndex]).length;
-        dataOutputStream.writeInt(complexByteArrayLength);
-        dataOutputStream.write(((byte[]) row[fieldIndex++]));
-      }
-
-      for (int counter = 0; counter < measureCount; counter++) {
-        if (null != row[fieldIndex]) {
-          dataOutputStream.write((byte) 1);
-          dataOutputStream.writeDouble((Double) NonDictionaryUtil.getMeasure(fieldIndex, row));
-        } else {
-          dataOutputStream.write((byte) 0);
-        }
-
-        fieldIndex++;
-      }
-
-    }
-  }
-
-  /**
-   * Below method will be used to write the sort temp file
-   *
-   * @param records
-   */
-  public void writeSortTempFile(Object[][] records) throws CarbonSortKeyAndGroupByException {
-    ByteArrayOutputStream blockDataArray = null;
-    DataOutputStream dataOutputStream = null;
-    int totalSize = 0;
-    int recordSize = 0;
-    try {
-      recordSize = (measureCount * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE) + (dimensionCount
-          * CarbonCommonConstants.INT_SIZE_IN_BYTE);
-      totalSize = records.length * recordSize;
-
-      blockDataArray = new ByteArrayOutputStream(totalSize);
-      dataOutputStream = new DataOutputStream(blockDataArray);
-
-      writeDataOutputStream(records, dataOutputStream, measureCount, dimensionCount,
-          noDictionaryCount, complexDimensionCount);
-      stream.writeInt(records.length);
-      byte[] byteArray = blockDataArray.toByteArray();
-      stream.writeInt(byteArray.length);
-      stream.write(byteArray);
-
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException(e);
-    } finally {
-      CarbonUtil.closeStreams(blockDataArray);
-      CarbonUtil.closeStreams(dataOutputStream);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/spliter/AbstractCarbonQueryExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/spliter/AbstractCarbonQueryExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/spliter/AbstractCarbonQueryExecutor.java
deleted file mode 100644
index 39d1234..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/spliter/AbstractCarbonQueryExecutor.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.spliter;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-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.util.CarbonUtil;
-
-public abstract class AbstractCarbonQueryExecutor {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractCarbonQueryExecutor.class.getName());
-  protected CarbonTable carbonTable;
-  protected QueryModel queryModel;
-  protected QueryExecutor queryExecutor;
-  protected Map<String, TaskBlockInfo> segmentMapping;
-
-  /**
-   * get executor and execute the query model.
-   *
-   * @param blockList
-   * @return
-   */
-  protected CarbonIterator<BatchResult> executeBlockList(List<TableBlockInfo> blockList)
-      throws QueryExecutionException, IOException {
-    queryModel.setTableBlockInfos(blockList);
-    this.queryExecutor = QueryExecutorFactory.getQueryExecutor(queryModel);
-    return queryExecutor.execute(queryModel);
-  }
-
-  /**
-   * 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.getFactTableName());
-    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.getFactTableName());
-    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
-   */
-  public void finish() {
-    try {
-      queryExecutor.finish();
-    } catch (QueryExecutionException e) {
-      LOGGER.error(e, "Problem while finish: ");
-    }
-    clearDictionaryFromQueryModel();
-  }
-
-  /**
-   * This method will clear the dictionary access count after its usage is complete so
-   * that column can be deleted form LRU cache whenever memory reaches threshold
-   */
-  private void clearDictionaryFromQueryModel() {
-    if (null != queryModel) {
-      Map<String, Dictionary> columnToDictionaryMapping = queryModel.getColumnToDictionaryMapping();
-      if (null != columnToDictionaryMapping) {
-        for (Map.Entry<String, Dictionary> entry : columnToDictionaryMapping.entrySet()) {
-          CarbonUtil.clearDictionaryCache(entry.getValue());
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/spliter/CarbonSplitExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/spliter/CarbonSplitExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/spliter/CarbonSplitExecutor.java
deleted file mode 100644
index 7b724ee..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/spliter/CarbonSplitExecutor.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.spliter;
-
-import java.io.IOException;
-import java.util.ArrayList;
-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;
-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.scan.executor.exception.QueryExecutionException;
-import org.apache.carbondata.core.scan.result.iterator.PartitionSpliterRawResultIterator;
-
-/**
- * Used to read carbon blocks when add/split partition
- */
-public class CarbonSplitExecutor extends AbstractCarbonQueryExecutor {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonSplitExecutor.class.getName());
-
-  public CarbonSplitExecutor(Map<String, TaskBlockInfo> segmentMapping, CarbonTable carbonTable) {
-    this.segmentMapping = segmentMapping;
-    this.carbonTable = carbonTable;
-  }
-
-  public List<PartitionSpliterRawResultIterator> processDataBlocks(String segmentId)
-      throws QueryExecutionException, IOException {
-    List<TableBlockInfo> list = null;
-    queryModel = prepareQueryModel(list);
-    List<PartitionSpliterRawResultIterator> resultList
-        = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    TaskBlockInfo taskBlockInfo = segmentMapping.get(segmentId);
-    Set<String> taskBlockListMapping = taskBlockInfo.getTaskSet();
-    for (String task : taskBlockListMapping) {
-      list = taskBlockInfo.getTableBlockInfoList(task);
-      LOGGER.info("for task -" + task + "-block size is -" + list.size());
-      queryModel.setTableBlockInfos(list);
-      resultList.add(new PartitionSpliterRawResultIterator(executeBlockList(list)));
-    }
-    return resultList;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/spliter/RowResultProcessor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/spliter/RowResultProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/spliter/RowResultProcessor.java
deleted file mode 100644
index 9316c9f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/spliter/RowResultProcessor.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.spliter;
-
-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.exception.CarbonDataWriterException;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
-import org.apache.carbondata.processing.spliter.exception.AlterPartitionSliceException;
-import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
-import org.apache.carbondata.processing.store.CarbonFactDataHandlerColumnar;
-import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
-import org.apache.carbondata.processing.store.CarbonFactHandler;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class RowResultProcessor {
-
-  private CarbonFactHandler dataHandler;
-  private SegmentProperties segmentProperties;
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(RowResultProcessor.class.getName());
-
-
-  public RowResultProcessor(CarbonTable carbonTable, CarbonLoadModel loadModel,
-      SegmentProperties segProp, String[] tempStoreLocation, Integer bucketId) {
-    CarbonDataProcessorUtil.createLocations(tempStoreLocation);
-    this.segmentProperties = segProp;
-    String tableName = carbonTable.getFactTableName();
-    CarbonFactDataHandlerModel carbonFactDataHandlerModel =
-        CarbonFactDataHandlerModel.getCarbonFactDataHandlerModel(loadModel, carbonTable,
-            segProp, tableName, tempStoreLocation);
-    CarbonDataFileAttributes carbonDataFileAttributes =
-        new CarbonDataFileAttributes(Integer.parseInt(loadModel.getTaskNo()),
-            loadModel.getFactTimeStamp());
-    carbonFactDataHandlerModel.setCarbonDataFileAttributes(carbonDataFileAttributes);
-    carbonFactDataHandlerModel.setBucketId(bucketId);
-    //Note: set compaction flow just to convert decimal type
-    carbonFactDataHandlerModel.setCompactionFlow(true);
-    dataHandler = new CarbonFactDataHandlerColumnar(carbonFactDataHandlerModel);
-  }
-
-  public boolean execute(List<Object[]> resultList) {
-    boolean processStatus;
-    boolean isDataPresent = false;
-
-    try {
-      if (!isDataPresent) {
-        dataHandler.initialise();
-        isDataPresent = true;
-      }
-      for (Object[] row: resultList) {
-        addRow(row);
-      }
-      if (isDataPresent)
-      {
-        this.dataHandler.finish();
-      }
-      processStatus = true;
-    } catch (AlterPartitionSliceException e) {
-      LOGGER.error(e, e.getMessage());
-      LOGGER.error("Exception in executing RowResultProcessor" + e.getMessage());
-      processStatus = false;
-    } finally {
-      try {
-        if (isDataPresent) {
-          this.dataHandler.closeHandler();
-        }
-      } catch (Exception e) {
-        LOGGER.error("Exception while closing the handler in RowResultProcessor" + e.getMessage());
-        processStatus = false;
-      }
-    }
-    return processStatus;
-  }
-
-  private void addRow(Object[] carbonTuple) throws AlterPartitionSliceException {
-    CarbonRow row = WriteStepRowUtil.fromMergerRow(carbonTuple, segmentProperties);
-    try {
-      this.dataHandler.addDataToStore(row);
-    } catch (CarbonDataWriterException e) {
-      throw new AlterPartitionSliceException("Exception in adding rows in RowResultProcessor", e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/spliter/exception/AlterPartitionSliceException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/spliter/exception/AlterPartitionSliceException.java b/processing/src/main/java/org/apache/carbondata/processing/spliter/exception/AlterPartitionSliceException.java
deleted file mode 100644
index 0e53a1f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/spliter/exception/AlterPartitionSliceException.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.spliter.exception;
-
-import java.util.Locale;
-
-public class AlterPartitionSliceException 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 AlterPartitionSliceException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public AlterPartitionSliceException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/splits/TableSplit.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/splits/TableSplit.java b/processing/src/main/java/org/apache/carbondata/processing/splits/TableSplit.java
new file mode 100644
index 0000000..c7d5dd8
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/splits/TableSplit.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.splits;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+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.processing.partition.Partition;
+
+import org.apache.hadoop.io.Writable;
+
+
+/**
+ * It represents one region server as one split.
+ */
+public class TableSplit implements Serializable, Writable {
+  private static final long serialVersionUID = -8058151330863145575L;
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(TableSplit.class.getName());
+  private List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  private Partition partition;
+
+  /**
+   * @return the locations
+   */
+  public List<String> getLocations() {
+    return locations;
+  }
+
+  /**
+   * @param locations the locations to set
+   */
+  public void setLocations(List<String> locations) {
+    this.locations = locations;
+  }
+
+  /**
+   * @return Returns the partitions.
+   */
+  public Partition getPartition() {
+    return partition;
+  }
+
+  /**
+   * @param partition The partitions to set.
+   */
+  public void setPartition(Partition partition) {
+    this.partition = partition;
+  }
+
+  @Override public void readFields(DataInput in) throws IOException {
+
+    int sizeLoc = in.readInt();
+    for (int i = 0; i < sizeLoc; i++) {
+      byte[] b = new byte[in.readInt()];
+      in.readFully(b);
+      locations.add(new String(b, Charset.defaultCharset()));
+    }
+
+    byte[] buf = new byte[in.readInt()];
+    in.readFully(buf);
+    ByteArrayInputStream bis = new ByteArrayInputStream(buf);
+    ObjectInputStream ois = new ObjectInputStream(bis);
+    try {
+      partition = (Partition) ois.readObject();
+    } catch (ClassNotFoundException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    ois.close();
+  }
+
+  @Override public void write(DataOutput out) throws IOException {
+
+    int sizeLoc = locations.size();
+    out.writeInt(sizeLoc);
+    for (int i = 0; i < sizeLoc; i++) {
+      byte[] bytes = locations.get(i).getBytes(Charset.defaultCharset());
+      out.writeInt(bytes.length);
+      out.write(bytes);
+    }
+
+    ByteArrayOutputStream bos = new ByteArrayOutputStream();
+
+    ObjectOutputStream obs = new ObjectOutputStream(bos);
+    obs.writeObject(partition);
+    obs.close();
+    byte[] byteArray = bos.toByteArray();
+    out.writeInt(byteArray.length);
+    out.write(byteArray);
+  }
+
+  public String toString() {
+    return partition.getUniqueID() + ' ' + locations;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataFileAttributes.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataFileAttributes.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataFileAttributes.java
index 0b606b0..b69815e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataFileAttributes.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataFileAttributes.java
@@ -17,20 +17,11 @@
 
 package org.apache.carbondata.processing.store;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-
 /**
  * This class contains attributes of file which are required to
  * construct file name like taskId, factTimeStamp
  */
 public class CarbonDataFileAttributes {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonDataFileAttributes.class.getName());
   /**
    * task Id which is unique for each spark task
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java
index 0fe922d..7a5cc11 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonDataWriterFactory.java
@@ -25,7 +25,7 @@ import org.apache.carbondata.processing.store.writer.v3.CarbonFactDataWriterImpl
 /**
  * Factory class to get the writer instance
  */
-public class CarbonDataWriterFactory {
+class CarbonDataWriterFactory {
 
   /**
    * static instance
@@ -56,7 +56,7 @@ public class CarbonDataWriterFactory {
    * @param carbonDataWriterVo writer vo object
    * @return writer instance
    */
-  public CarbonFactDataWriter<?> getFactDataWriter(final ColumnarFormatVersion version,
+  public CarbonFactDataWriter getFactDataWriter(final ColumnarFormatVersion version,
       final CarbonDataWriterVo carbonDataWriterVo) {
     switch (version) {
       case V1:

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 c4a5fc5..2c275bf 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
@@ -50,7 +50,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.newflow.sort.SortScopeOptions;
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
 import org.apache.carbondata.processing.store.file.FileManager;
 import org.apache.carbondata.processing.store.file.IFileManagerComposite;
 import org.apache.carbondata.processing.store.writer.CarbonDataWriterVo;
@@ -88,10 +88,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
    * once this size of input is reached
    */
   private int pageSize;
-  /**
-   * keyBlockHolder
-   */
-  private CarbonKeyBlockHolder[] keyBlockHolder;
 
   // This variable is true if it is dictionary dimension and its cardinality is lower than
   // property of CarbonCommonConstants.HIGH_CARDINALITY_VALUE
@@ -455,7 +451,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       this.dataWriter.closeWriter();
     }
     this.dataWriter = null;
-    this.keyBlockHolder = null;
   }
 
   /**
@@ -488,15 +483,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       //than below splitter will return column as {0,1,2}{3}{4}{5}
       ColumnarSplitter columnarSplitter = model.getSegmentProperties().getFixedLengthKeySplitter();
       System.arraycopy(columnarSplitter.getBlockKeySize(), 0, keyBlockSize, 0, noOfColStore);
-      this.keyBlockHolder =
-          new CarbonKeyBlockHolder[columnarSplitter.getBlockKeySize().length];
-    } else {
-      this.keyBlockHolder = new CarbonKeyBlockHolder[0];
-    }
-
-    for (int i = 0; i < keyBlockHolder.length; i++) {
-      this.keyBlockHolder[i] = new CarbonKeyBlockHolder(pageSize);
-      this.keyBlockHolder[i].resetCounter();
     }
 
     // agg type
@@ -567,7 +553,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
    *
    * @return data writer instance
    */
-  private CarbonFactDataWriter<?> getFactDataWriter() {
+  private CarbonFactDataWriter getFactDataWriter() {
     return CarbonDataWriterFactory.getInstance()
         .getFactDataWriter(version, getDataWriterVo());
   }
@@ -680,10 +666,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       return tablePage;
     }
 
-    /**
-     * @param encodedTablePage
-     * @param index
-     */
     public synchronized void put(TablePage tablePage, int index) {
       tablePages[index] = tablePage;
       // notify the consumer thread when index at which object is to be inserted

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 544a26a..2c346b2 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
@@ -40,10 +40,10 @@ 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;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
-import org.apache.carbondata.processing.newflow.sort.SortScopeOptions;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 // This class contains all the data required for processing and writing the carbon data

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/CarbonKeyBlockHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonKeyBlockHolder.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonKeyBlockHolder.java
deleted file mode 100644
index 898917b..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonKeyBlockHolder.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.store;
-
-public class CarbonKeyBlockHolder {
-  private byte[][] keyBlock;
-
-  private int counter;
-
-  public CarbonKeyBlockHolder(int size) {
-    keyBlock = new byte[size][];
-  }
-
-  public void addRowToBlock(int index, byte[] keyBlock) {
-    this.keyBlock[index] = keyBlock;
-    counter++;
-  }
-
-  public byte[][] getKeyBlock() {
-    if (counter < keyBlock.length) {
-      byte[][] temp = new byte[counter][];
-      System.arraycopy(keyBlock, 0, temp, 0, counter);
-      return temp;
-    }
-    return keyBlock;
-  }
-
-  public void resetCounter() {
-    counter = 0;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
deleted file mode 100644
index 48227d1..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
+++ /dev/null
@@ -1,313 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.store;
-
-import java.io.File;
-import java.io.FileFilter;
-import java.util.AbstractQueue;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.PriorityQueue;
-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.exception.CarbonDataWriterException;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortTempFileChunkHolder;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SingleThreadFinalSortFilesMerger.class.getName());
-
-  /**
-   * lockObject
-   */
-  private static final Object LOCKOBJECT = new Object();
-
-  /**
-   * fileCounter
-   */
-  private int fileCounter;
-
-  /**
-   * fileBufferSize
-   */
-  private int fileBufferSize;
-
-  /**
-   * recordHolderHeap
-   */
-  private AbstractQueue<SortTempFileChunkHolder> recordHolderHeapLocal;
-
-  /**
-   * tableName
-   */
-  private String tableName;
-
-  /**
-   * measureCount
-   */
-  private int measureCount;
-
-  /**
-   * dimensionCount
-   */
-  private int dimensionCount;
-
-  /**
-   * measure count
-   */
-  private int noDictionaryCount;
-
-  /**
-   * complexDimensionCount
-   */
-  private int complexDimensionCount;
-
-  /**
-   * tempFileLocation
-   */
-  private String[] tempFileLocation;
-
-  private DataType[] measureDataType;
-
-  /**
-   * below code is to check whether dimension
-   * is of no dictionary type or not
-   */
-  private boolean[] isNoDictionaryColumn;
-
-  private boolean[] isNoDictionarySortColumn;
-
-  public SingleThreadFinalSortFilesMerger(String[] tempFileLocation, String tableName,
-      int dimensionCount, int complexDimensionCount, int measureCount, int noDictionaryCount,
-      DataType[] type, boolean[] isNoDictionaryColumn, boolean[] isNoDictionarySortColumn) {
-    this.tempFileLocation = tempFileLocation;
-    this.tableName = tableName;
-    this.dimensionCount = dimensionCount;
-    this.complexDimensionCount = complexDimensionCount;
-    this.measureCount = measureCount;
-    this.measureDataType = type;
-    this.noDictionaryCount = noDictionaryCount;
-    this.isNoDictionaryColumn = isNoDictionaryColumn;
-    this.isNoDictionarySortColumn = isNoDictionarySortColumn;
-  }
-
-  /**
-   * This method will be used to merger the merged files
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  public void startFinalMerge() throws CarbonDataWriterException {
-    List<File> filesToMerge = getFilesToMergeSort();
-    if (filesToMerge.size() == 0)
-    {
-      LOGGER.info("No file to merge in final merge stage");
-      return;
-    }
-
-    startSorting(filesToMerge);
-  }
-
-  private List<File> getFilesToMergeSort() {
-    FileFilter fileFilter = new FileFilter() {
-      public boolean accept(File pathname) {
-        return pathname.getName().startsWith(tableName);
-      }
-    };
-
-    // get all the merged files
-    List<File> files = new ArrayList<File>(tempFileLocation.length);
-    for (String tempLoc : tempFileLocation)
-    {
-      File[] subFiles = new File(tempLoc).listFiles(fileFilter);
-      if (null != subFiles && subFiles.length > 0)
-      {
-        files.addAll(Arrays.asList(subFiles));
-      }
-    }
-
-    return files;
-  }
-
-  /**
-   * Below method will be used to start storing process This method will get
-   * all the temp files present in sort temp folder then it will create the
-   * record holder heap and then it will read first record from each file and
-   * initialize the heap
-   *
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private void startSorting(List<File> files) throws CarbonDataWriterException {
-    this.fileCounter = files.size();
-    if (fileCounter == 0) {
-      LOGGER.info("No files to merge sort");
-      return;
-    }
-    this.fileBufferSize = CarbonDataProcessorUtil
-        .getFileBufferSize(this.fileCounter, CarbonProperties.getInstance(),
-            CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-    LOGGER.info("Number of temp file: " + this.fileCounter);
-
-    LOGGER.info("File Buffer Size: " + this.fileBufferSize);
-
-    // create record holder heap
-    createRecordHolderQueue();
-
-    // iterate over file list and create chunk holder and add to heap
-    LOGGER.info("Started adding first record from each file");
-    int maxThreadForSorting = 0;
-    try {
-      maxThreadForSorting = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD,
-              CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE));
-    } catch (NumberFormatException e) {
-      maxThreadForSorting =
-          Integer.parseInt(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE);
-    }
-    ExecutorService service = Executors.newFixedThreadPool(maxThreadForSorting);
-
-    for (final File tempFile : files) {
-
-      Runnable runnable = new Runnable() {
-        @Override public void run() {
-
-            // create chunk holder
-            SortTempFileChunkHolder sortTempFileChunkHolder =
-                new SortTempFileChunkHolder(tempFile, dimensionCount, complexDimensionCount,
-                    measureCount, fileBufferSize, noDictionaryCount, measureDataType,
-                    isNoDictionaryColumn, isNoDictionarySortColumn);
-          try {
-            // initialize
-            sortTempFileChunkHolder.initialize();
-            sortTempFileChunkHolder.readRow();
-          } catch (CarbonSortKeyAndGroupByException ex) {
-            LOGGER.error(ex);
-          }
-
-          synchronized (LOCKOBJECT) {
-            recordHolderHeapLocal.add(sortTempFileChunkHolder);
-          }
-        }
-      };
-      service.execute(runnable);
-    }
-    service.shutdown();
-
-    try {
-      service.awaitTermination(2, TimeUnit.HOURS);
-    } catch (Exception e) {
-      throw new CarbonDataWriterException(e.getMessage(), e);
-    }
-
-    LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
-  }
-
-  /**
-   * This method will be used to create the heap which will be used to hold
-   * the chunk of data
-   */
-  private void createRecordHolderQueue() {
-    // creating record holder heap
-    this.recordHolderHeapLocal = new PriorityQueue<SortTempFileChunkHolder>(fileCounter);
-  }
-
-  /**
-   * This method will be used to get the sorted row
-   *
-   * @return sorted row
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  public Object[] next() {
-    return getSortedRecordFromFile();
-  }
-
-  /**
-   * This method will be used to get the sorted record from file
-   *
-   * @return sorted record sorted record
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  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
-    // be based on comparator we are passing the heap
-    // when will call poll it will always delete root of the tree and then
-    // it does trickel down operation complexity is log(n)
-    SortTempFileChunkHolder poll = this.recordHolderHeapLocal.poll();
-
-    // get the row from chunk
-    row = poll.getRow();
-
-    // check if there no entry present
-    if (!poll.hasNext()) {
-      // if chunk is empty then close the stream
-      poll.closeStream();
-
-      // change the file counter
-      --this.fileCounter;
-
-      // reaturn row
-      return row;
-    }
-
-    // read new row
-    try {
-      poll.readRow();
-    } catch (CarbonSortKeyAndGroupByException e) {
-      throw new CarbonDataWriterException(e.getMessage(), e);
-    }
-
-    // add to heap
-    this.recordHolderHeapLocal.add(poll);
-
-    // return row
-    return row;
-  }
-
-  /**
-   * This method will be used to check whether any more element is present or
-   * not
-   *
-   * @return more element is present
-   */
-  public boolean hasNext() {
-    return this.fileCounter > 0;
-  }
-
-  public void clear() {
-    if (null != recordHolderHeapLocal) {
-      recordHolderHeapLocal = null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColGroupDataHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColGroupDataHolder.java b/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColGroupDataHolder.java
deleted file mode 100644
index a2e22c2..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColGroupDataHolder.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.store.colgroup;
-
-
-/**
- * This will hold column group data.
- */
-public class ColGroupDataHolder implements DataHolder {
-
-  private int noOfRecords;
-
-  /**
-   * colGrpData[row no][data]
-   */
-  private byte[][] colGrpData;
-
-  /**
-   * This will have min max value of each chunk
-   */
-  private ColGroupMinMax colGrpMinMax;
-
-  /**
-   * each row size of this column group block
-   */
-  private int keyBlockSize;
-
-  /**
-   * @param keyBlockSize
-   * @param noOfRecords
-   * @param colGrpMinMax
-   */
-  public ColGroupDataHolder(int keyBlockSize,
-       int noOfRecords,ColGroupMinMax colGrpMinMax) {
-    this.noOfRecords = noOfRecords;
-    this.keyBlockSize = keyBlockSize;
-    this.colGrpMinMax = colGrpMinMax;
-    colGrpData = new byte[noOfRecords][];
-  }
-
-  @Override public void addData(byte[] rowsData, int rowIndex) {
-    colGrpData[rowIndex] = rowsData;
-    colGrpMinMax.add(rowsData);
-  }
-
-  /**
-   * this will return min of each chunk
-   *
-   * @return
-   */
-  public byte[] getMin() {
-    return colGrpMinMax.getMin();
-  }
-
-  /**
-   * this will return max of each chunk
-   *
-   * @return
-   */
-  public byte[] getMax() {
-    return colGrpMinMax.getMax();
-  }
-
-  /**
-   * Return size of this column group block
-   *
-   * @return
-   */
-  public int getKeyBlockSize() {
-    return keyBlockSize;
-  }
-
-  @Override public byte[][] getData() {
-    return colGrpData;
-  }
-
-  /**
-   * return total size required by this block
-   *
-   * @return
-   */
-  public int getTotalSize() {
-    return noOfRecords * keyBlockSize;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColGroupMinMax.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColGroupMinMax.java b/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColGroupMinMax.java
deleted file mode 100644
index b3d11f2..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColGroupMinMax.java
+++ /dev/null
@@ -1,215 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.store.colgroup;
-
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-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.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.util.ByteUtil;
-
-/**
- * it gives min max of each column of column group
- */
-public class ColGroupMinMax {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ColGroupMinMax.class.getName());
-
-  /**
-   * key generator
-   */
-  private KeyGenerator keyGenerator;
-
-  /**
-   * no of column in column group
-   */
-  private int noOfCol;
-
-  /**
-   * min value of each column
-   */
-  private byte[][] min;
-
-  /**
-   * max value of each column
-   */
-  private byte[][] max;
-
-  /**
-   * mask byte range
-   */
-  private int[][] maskByteRange;
-
-  /**
-   * max keys
-   */
-  private byte[][] maxKeys;
-
-  public ColGroupMinMax(SegmentProperties segmentProperties, int colGroupId) {
-    this.keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGroupId);
-    this.noOfCol = segmentProperties.getNoOfColumnsInColumnGroup(colGroupId);
-    min = new byte[noOfCol][];
-    max = new byte[noOfCol][];
-    initialise();
-  }
-
-  /**
-   * @param mdkey
-   */
-  public void add(byte[] mdkey) {
-    for (int i = 0; i < noOfCol; i++) {
-      byte[] col = getMaskedKey(mdkey, maskByteRange[i], maxKeys[i]);
-      setMin(col, i);
-      setMax(col, i);
-    }
-  }
-
-  /**
-   * Below method will be used to get the masked key
-   *
-   * @param data
-   * @return maskedKey
-   */
-  private byte[] getMaskedKey(byte[] data, int[] maskByteRange, byte[] maxKey) {
-    int keySize = maskByteRange.length;
-    byte[] maskedKey = new byte[keySize];
-    int counter = 0;
-    int byteRange = 0;
-    for (int i = 0; i < keySize; i++) {
-      byteRange = maskByteRange[i];
-      maskedKey[counter++] = (byte) (data[byteRange] & maxKey[byteRange]);
-    }
-    return maskedKey;
-  }
-
-  /**
-   * intitialising data required for min max calculation
-   */
-  private void initialise() {
-    try {
-      maskByteRange = new int[noOfCol][];
-      maxKeys = new byte[noOfCol][];
-      for (int i = 0; i < noOfCol; i++) {
-        maskByteRange[i] = getMaskByteRange(i);
-        // generating maxkey
-        long[] maxKey = new long[noOfCol];
-        maxKey[i] = Long.MAX_VALUE;
-        maxKeys[i] = keyGenerator.generateKey(maxKey);
-      }
-    } catch (KeyGenException e) {
-      LOGGER.error(e, "Key generation failed while evaulating column group min max");
-    }
-
-  }
-
-  /**
-   * get range for given column in generated md key
-   *
-   * @param col : column
-   * @return maskByteRange
-   */
-  private int[] getMaskByteRange(int col) {
-    Set<Integer> integers = new HashSet<>();
-    int[] range = keyGenerator.getKeyByteOffsets(col);
-    for (int j = range[0]; j <= range[1]; j++) {
-      integers.add(j);
-    }
-    int[] byteIndexs = new int[integers.size()];
-    int j = 0;
-    for (Iterator<Integer> iterator = integers.iterator(); iterator.hasNext(); ) {
-      Integer integer = (Integer) iterator.next();
-      byteIndexs[j++] = integer.intValue();
-    }
-    return byteIndexs;
-  }
-
-  /**
-   * set min value of given column
-   *
-   * @param colData
-   * @param column
-   */
-  private void setMin(byte[] colData, int column) {
-
-    if (null == min[column]) {
-      min[column] = colData;
-    } else {
-      if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(colData, min[column]) < 0) {
-        min[column] = colData;
-      }
-    }
-  }
-
-  /**
-   * set max value of given column
-   *
-   * @param colData
-   * @param column
-   */
-  private void setMax(byte[] colData, int column) {
-    if (null == max[column]) {
-      max[column] = colData;
-    } else {
-      if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(colData, max[column]) > 0) {
-        max[column] = colData;
-      }
-
-    }
-  }
-
-  /**
-   * Get min value of block
-   *
-   * @return min value of block
-   */
-  public byte[] getMin() {
-    int size = 0;
-    for (int i = 0; i < noOfCol; i++) {
-      size += min[i].length;
-    }
-    ByteBuffer bb = ByteBuffer.allocate(size);
-    for (int i = 0; i < noOfCol; i++) {
-      bb.put(min[i]);
-    }
-    return bb.array();
-  }
-
-  /**
-   * get max value of block
-   *
-   * @return max value of block
-   */
-  public byte[] getMax() {
-    int size = 0;
-    for (int i = 0; i < noOfCol; i++) {
-      size += max[i].length;
-    }
-    ByteBuffer bb = ByteBuffer.allocate(size);
-    for (int i = 0; i < noOfCol; i++) {
-      bb.put(max[i]);
-    }
-    return bb.array();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColumnDataHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColumnDataHolder.java b/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColumnDataHolder.java
deleted file mode 100644
index 8caf339..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/ColumnDataHolder.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.processing.store.colgroup;
-
-/**
- * Store mdkey data for each column block
- */
-public class ColumnDataHolder implements DataHolder {
-
-  private byte[][] data;
-
-  public ColumnDataHolder(int noOfRow) {
-    data = new byte[noOfRow][];
-  }
-
-  @Override public void addData(byte[] rowRecord, int rowIndex) {
-    data[rowIndex] = rowRecord;
-  }
-
-  @Override public byte[][] getData() {
-    return data;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/DataHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/DataHolder.java b/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/DataHolder.java
deleted file mode 100644
index 3b1ee81..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/store/colgroup/DataHolder.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.processing.store.colgroup;
-
-/**
- * Hold complete data for a leaf node
- */
-public interface DataHolder {
-
-  /**
-   * add row to holder
-   *
-   * @param rowRecord: row data
-   * @param rowIndex   : row number
-   */
-  public void addData(byte[] rowRecord, int rowIndex);
-
-  /**
-   * return the data when required
-   *
-   * @return all data
-   */
-  public byte[][] getData();
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 ec42596..acb3b3b 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
@@ -64,7 +64,7 @@ import org.apache.carbondata.processing.store.file.FileData;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.hadoop.io.IOUtils;
 
-public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<T> {
+public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(AbstractFactDataWriter.class.getName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
index e195d10..3a2fa1c 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonFactDataWriter.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.processing.store.TablePage;
 
-public interface CarbonFactDataWriter<T> {
+public interface CarbonFactDataWriter {
 
   /**
    * write a encoded table page


[44/50] [abbrv] carbondata git commit: [CARBONDATA-1514] Sort Column Property is not getting added in case of alter operation

Posted by ja...@apache.org.
[CARBONDATA-1514] Sort Column Property is not getting added in case of alter operation

Problem:: Sort Column Property is not getting added in case of alter operation
Solution: Need to add sort column if use has added in alter operation

This closes #1382


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

Branch: refs/heads/streaming_ingest
Commit: a7c36ef4cb3b2652748e71b782fc8475502efa21
Parents: a734add
Author: kumarvishal <ku...@gmail.com>
Authored: Mon Sep 25 17:45:10 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 5 16:43:15 2017 +0530

----------------------------------------------------------------------
 .../command/carbonTableSchemaCommon.scala         | 18 +++++++++++++++---
 1 file changed, 15 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/a7c36ef4/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 ebd1c6c..1a39741 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
@@ -189,6 +189,14 @@ class AlterTableColumnSchemaGenerator(
 
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
 
+  def isSortColumn(columnName: String): Boolean = {
+    val sortColumns = alterTableModel.tableProperties.get("sort_columns")
+    if(sortColumns.isDefined) {
+      sortColumns.get.contains(columnName)
+    } else {
+      true
+    }
+  }
   def process: Seq[ColumnSchema] = {
     val tableSchema = tableInfo.getFactTable
     val tableCols = tableSchema.getListOfColumns.asScala
@@ -208,7 +216,8 @@ class AlterTableColumnSchemaGenerator(
         -1,
         field.precision,
         field.scale,
-        field.schemaOrdinal + existingColsSize)
+        field.schemaOrdinal + existingColsSize,
+        isSortColumn(field.name.getOrElse(field.column)))
       allColumns ++= Seq(columnSchema)
       newCols ++= Seq(columnSchema)
     })
@@ -225,7 +234,8 @@ class AlterTableColumnSchemaGenerator(
         -1,
         field.precision,
         field.scale,
-        field.schemaOrdinal + existingColsSize)
+        field.schemaOrdinal + existingColsSize,
+        false)
       allColumns ++= Seq(columnSchema)
       newCols ++= Seq(columnSchema)
     })
@@ -291,7 +301,8 @@ class AlterTableColumnSchemaGenerator(
 
   private def getColumnSchema(dataType: DataType, colName: String, isCol: Boolean,
       encoders: java.util.List[Encoding], isDimensionCol: Boolean,
-      colGroup: Integer, precision: Integer, scale: Integer, schemaOrdinal: Int): ColumnSchema = {
+      colGroup: Integer, precision: Integer, scale: Integer, schemaOrdinal: Int,
+      isSortColumn: Boolean): ColumnSchema = {
     val columnSchema = new ColumnSchema()
     columnSchema.setDataType(dataType)
     columnSchema.setColumnName(colName)
@@ -319,6 +330,7 @@ class AlterTableColumnSchemaGenerator(
     columnSchema.setScale(scale)
     columnSchema.setSchemaOrdinal(schemaOrdinal)
     columnSchema.setUseInvertedIndex(isDimensionCol)
+    columnSchema.setSortColumn(isSortColumn)
     columnSchema
   }
 }


[36/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/row/CarbonRowBatch.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/row/CarbonRowBatch.java b/processing/src/main/java/org/apache/carbondata/processing/loading/row/CarbonRowBatch.java
new file mode 100644
index 0000000..f5f112c
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/row/CarbonRowBatch.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.row;
+
+import java.util.NoSuchElementException;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+
+
+/**
+ * Batch of rows.
+ */
+public class CarbonRowBatch extends CarbonIterator<CarbonRow> {
+
+  private CarbonRow[] rowBatch;
+
+  private int size = 0;
+
+  private int index = 0;
+
+  public CarbonRowBatch(int batchSize) {
+    this.rowBatch = new CarbonRow[batchSize];
+  }
+
+  public void addRow(CarbonRow carbonRow) {
+    rowBatch[size++] = carbonRow;
+  }
+
+  public int getSize() {
+    return size;
+  }
+
+  @Override public boolean hasNext() {
+    return index < size;
+  }
+
+  @Override
+  public CarbonRow next() throws NoSuchElementException {
+    if (hasNext()) {
+      return rowBatch[index++];
+    }
+    throw new NoSuchElementException("no more elements to iterate");
+  }
+
+  @Override public void remove() {
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/row/CarbonSortBatch.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/row/CarbonSortBatch.java b/processing/src/main/java/org/apache/carbondata/processing/loading/row/CarbonSortBatch.java
new file mode 100644
index 0000000..5d39145
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/row/CarbonSortBatch.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.processing.loading.row;
+
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.processing.loading.sort.unsafe.merger.UnsafeSingleThreadFinalSortFilesMerger;
+
+/**
+ * Batch of sorted rows which are ready to be processed by
+ */
+public class CarbonSortBatch extends CarbonRowBatch {
+
+  private UnsafeSingleThreadFinalSortFilesMerger iterator;
+
+  public CarbonSortBatch(UnsafeSingleThreadFinalSortFilesMerger iterator) {
+    super(0);
+    this.iterator = iterator;
+  }
+
+  @Override public boolean hasNext() {
+    return iterator.hasNext();
+  }
+
+  @Override public CarbonRow next() {
+    return new CarbonRow(iterator.next());
+  }
+
+  @Override public void close() {
+    iterator.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/AbstractMergeSorter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/AbstractMergeSorter.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/AbstractMergeSorter.java
new file mode 100644
index 0000000..550fe70
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/AbstractMergeSorter.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.sort;
+
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.sort.impl.ThreadStatusObserver;
+
+/**
+ * The class defines the common methods used in across various type of sort
+ */
+public abstract class AbstractMergeSorter implements Sorter {
+  /**
+   * instance of thread status observer
+   */
+  protected ThreadStatusObserver threadStatusObserver;
+
+  /**
+   * Below method will be used to check error in exception
+   */
+  public void checkError() {
+    if (threadStatusObserver.getThrowable() != null) {
+      if (threadStatusObserver.getThrowable() instanceof CarbonDataLoadingException) {
+        throw (CarbonDataLoadingException) threadStatusObserver.getThrowable();
+      } else {
+        throw new CarbonDataLoadingException(threadStatusObserver.getThrowable());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortScopeOptions.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortScopeOptions.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortScopeOptions.java
new file mode 100644
index 0000000..23179fa
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortScopeOptions.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.util.CarbonUtil;
+
+/**
+ * Sort scope options
+ */
+public class SortScopeOptions {
+
+  public static SortScope getSortScope(String sortScope) {
+    if (sortScope == null) {
+      sortScope = CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT;
+    }
+    switch (sortScope.toUpperCase()) {
+      case "BATCH_SORT":
+        return SortScope.BATCH_SORT;
+      case "LOCAL_SORT":
+        return SortScope.LOCAL_SORT;
+      case "GLOBAL_SORT":
+        return SortScope.GLOBAL_SORT;
+      case "NO_SORT":
+        return SortScope.NO_SORT;
+      default:
+        return SortScope.LOCAL_SORT;
+    }
+  }
+
+  public static boolean isValidSortOption(String sortScope) {
+    return CarbonUtil.isValidSortOption(sortScope);
+  }
+
+  public enum SortScope {
+    NO_SORT, BATCH_SORT, LOCAL_SORT, GLOBAL_SORT;
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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..9665487
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SortStepRowUtil.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.processing.loading.sort;
+
+import org.apache.carbondata.core.util.NonDictionaryUtil;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+
+public class SortStepRowUtil {
+  public static Object[] convertRow(Object[] data, SortParameters parameters) {
+    int measureCount = parameters.getMeasureColCount();
+    int dimensionCount = parameters.getDimColCount();
+    int complexDimensionCount = parameters.getComplexDimColCount();
+    int noDictionaryCount = parameters.getNoDictionaryCount();
+    boolean[] isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
+
+    // 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 allCount = 0;
+    int[] dim = new int[dimensionCount];
+    byte[][] nonDicArray = new byte[noDictionaryCount + complexDimensionCount][];
+    Object[] measures = new Object[measureCount];
+    try {
+      // read dimension values
+      for (int i = 0; i < isNoDictionaryDimensionColumn.length; i++) {
+        if (isNoDictionaryDimensionColumn[i]) {
+          nonDicArray[nonDicIndex++] = (byte[]) data[i];
+        } else {
+          dim[index++] = (int) data[allCount];
+        }
+        allCount++;
+      }
+
+      for (int i = 0; i < complexDimensionCount; i++) {
+        nonDicArray[nonDicIndex++] = (byte[]) data[allCount];
+        allCount++;
+      }
+
+      index = 0;
+
+      // read measure values
+      for (int i = 0; i < measureCount; i++) {
+        measures[index++] = data[allCount];
+        allCount++;
+      }
+
+      NonDictionaryUtil.prepareOutObj(holder, dim, nonDicArray, 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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/Sorter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/Sorter.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/Sorter.java
new file mode 100644
index 0000000..9a47e50
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/Sorter.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.sort;
+
+import java.util.Iterator;
+
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+
+/**
+ * This interface sorts all the data of iterators.
+ * The life cycle of this interface is initialize -> sort -> close
+ */
+public interface Sorter {
+
+  /**
+   * Initialize sorter with sort parameters.
+   *
+   * @param sortParameters
+   */
+  void initialize(SortParameters sortParameters);
+
+  /**
+   * Sorts the data of all iterators, this iterators can be
+   * read parallely depends on implementation.
+   *
+   * @param iterators array of iterators to read data.
+   * @return
+   * @throws CarbonDataLoadingException
+   */
+  Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
+      throws CarbonDataLoadingException;
+
+  /**
+   * Close resources
+   */
+  void close();
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..a8f0282
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/SorterFactory.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.processing.loading.sort;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+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.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.UnsafeBatchParallelReadMergeSorterImpl;
+import org.apache.carbondata.processing.loading.sort.impl.UnsafeParallelReadMergeSorterImpl;
+import org.apache.carbondata.processing.loading.sort.impl.UnsafeParallelReadMergeSorterWithBucketingImpl;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+public class SorterFactory {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(SorterFactory.class.getName());
+
+  public static Sorter createSorter(CarbonDataLoadConfiguration configuration, AtomicLong counter) {
+    boolean offheapsort = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
+            CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT));
+    SortScopeOptions.SortScope sortScope = CarbonDataProcessorUtil.getSortScope(configuration);
+    Sorter sorter;
+    if (offheapsort) {
+      if (configuration.getBucketingInfo() != null) {
+        sorter = new UnsafeParallelReadMergeSorterWithBucketingImpl(configuration.getDataFields(),
+            configuration.getBucketingInfo());
+      } else {
+        sorter = new UnsafeParallelReadMergeSorterImpl(counter);
+      }
+    } else {
+      if (configuration.getBucketingInfo() != null) {
+        sorter =
+            new ParallelReadMergeSorterWithBucketingImpl(counter, configuration.getBucketingInfo());
+      } else {
+        sorter = new ParallelReadMergeSorterImpl(counter);
+      }
+    }
+    if (sortScope.equals(SortScopeOptions.SortScope.BATCH_SORT)) {
+      if (configuration.getBucketingInfo() == null) {
+        sorter = new UnsafeBatchParallelReadMergeSorterImpl(counter);
+      } else {
+        LOGGER.warn(
+            "Batch sort is not enabled in case of bucketing. Falling back to " + sorter.getClass()
+                .getName());
+      }
+    }
+    return sorter;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..6e43fcb
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterImpl.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.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.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.
+ */
+public class ParallelReadMergeSorterImpl extends AbstractMergeSorter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ParallelReadMergeSorterImpl.class.getName());
+
+  private SortParameters sortParameters;
+
+  private SortIntermediateFileMerger intermediateFileMerger;
+
+  private SingleThreadFinalSortFilesMerger finalMerger;
+
+  private AtomicLong rowCounter;
+
+  public ParallelReadMergeSorterImpl(AtomicLong rowCounter) {
+    this.rowCounter = rowCounter;
+  }
+
+  @Override
+  public void initialize(SortParameters sortParameters) {
+    this.sortParameters = sortParameters;
+    intermediateFileMerger = new SortIntermediateFileMerger(sortParameters);
+    String[] storeLocations =
+        CarbonDataProcessorUtil.getLocalDataFolderLocation(
+            sortParameters.getDatabaseName(), sortParameters.getTableName(),
+            String.valueOf(sortParameters.getTaskNo()), sortParameters.getPartitionID(),
+            sortParameters.getSegmentId() + "", false, false);
+    // Set the data file location
+    String[] dataFolderLocations = CarbonDataProcessorUtil.arrayAppend(storeLocations,
+        File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
+    finalMerger =
+        new SingleThreadFinalSortFilesMerger(dataFolderLocations, sortParameters.getTableName(),
+            sortParameters.getDimColCount(),
+            sortParameters.getComplexDimColCount(), sortParameters.getMeasureColCount(),
+            sortParameters.getNoDictionaryCount(), sortParameters.getMeasureDataType(),
+            sortParameters.getNoDictionaryDimnesionColumn(),
+            sortParameters.getNoDictionarySortColumn());
+  }
+
+  @Override
+  public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
+      throws CarbonDataLoadingException {
+    SortDataRows sortDataRow = new SortDataRows(sortParameters, intermediateFileMerger);
+    final int batchSize = CarbonProperties.getInstance().getBatchSize();
+    try {
+      sortDataRow.initialize();
+    } catch (CarbonSortKeyAndGroupByException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
+    this.threadStatusObserver = new ThreadStatusObserver(executorService);
+
+    try {
+      for (int i = 0; i < iterators.length; i++) {
+        executorService.execute(
+            new SortIteratorThread(iterators[i], sortDataRow, batchSize, rowCounter,
+                threadStatusObserver));
+      }
+      executorService.shutdown();
+      executorService.awaitTermination(2, TimeUnit.DAYS);
+      processRowToNextStep(sortDataRow, sortParameters);
+    } catch (Exception e) {
+      checkError();
+      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
+    }
+    checkError();
+    try {
+      intermediateFileMerger.finish();
+      intermediateFileMerger = null;
+      finalMerger.startFinalMerge();
+    } catch (CarbonDataWriterException e) {
+      throw new CarbonDataLoadingException(e);
+    } catch (CarbonSortKeyAndGroupByException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+
+    // Creates the iterator to read from merge sorter.
+    Iterator<CarbonRowBatch> batchIterator = new CarbonIterator<CarbonRowBatch>() {
+
+      @Override
+      public boolean hasNext() {
+        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;
+      }
+    };
+    return new Iterator[] { batchIterator };
+  }
+
+  @Override public void close() {
+    if (intermediateFileMerger != null) {
+      intermediateFileMerger.close();
+    }
+  }
+
+  /**
+   * Below method will be used to process data to next step
+   */
+  private boolean processRowToNextStep(SortDataRows sortDataRows, SortParameters parameters)
+      throws CarbonDataLoadingException {
+    if (null == sortDataRows) {
+      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 {
+      // start sorting
+      sortDataRows.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);
+    }
+  }
+
+  /**
+   * 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 Object[][] buffer;
+
+    private AtomicLong rowCounter;
+
+    private ThreadStatusObserver observer;
+
+    public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortDataRows sortDataRows,
+        int batchSize, AtomicLong rowCounter, ThreadStatusObserver observer) {
+      this.iterator = iterator;
+      this.sortDataRows = sortDataRows;
+      this.buffer = new Object[batchSize][];
+      this.rowCounter = rowCounter;
+      this.observer = 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) {
+              buffer[i++] = row.getData();
+            }
+          }
+          if (i > 0) {
+            sortDataRows.addRowBatch(buffer, i);
+            rowCounter.getAndAdd(i);
+          }
+        }
+      } catch (Exception e) {
+        LOGGER.error(e);
+        observer.notifyFailed(e);
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..51db3a0
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
@@ -0,0 +1,276 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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()), bucketId,
+            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.getDimColCount(), sortParameters.getComplexDimColCount(),
+            sortParameters.getMeasureColCount(), sortParameters.getNoDictionaryCount(),
+            sortParameters.getMeasureDataType(), sortParameters.getNoDictionaryDimnesionColumn(),
+            this.sortParameters.getNoDictionarySortColumn());
+  }
+
+  @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.getPartitionID(), 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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ThreadStatusObserver.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ThreadStatusObserver.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ThreadStatusObserver.java
new file mode 100644
index 0000000..ed35a96
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/ThreadStatusObserver.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.sort.impl;
+
+import java.util.concurrent.ExecutorService;
+
+public class ThreadStatusObserver {
+
+  /**
+   * lock object
+   */
+  private Object lock = new Object();
+
+  private ExecutorService executorService;
+
+  private Throwable throwable;
+
+  public ThreadStatusObserver(ExecutorService executorService) {
+    this.executorService = executorService;
+  }
+
+  public void notifyFailed(Throwable throwable) {
+    // Only the first failing thread should call for shutting down the executor service and
+    // should assign the throwable object else the actual cause for failure can be overridden as
+    // all the running threads will throw interrupted exception on calling shutdownNow and
+    // will override the throwable object
+    if (null == this.throwable) {
+      synchronized (lock) {
+        if (null == this.throwable) {
+          executorService.shutdownNow();
+          this.throwable = throwable;
+        }
+      }
+    }
+  }
+
+  public Throwable getThrowable() {
+    return throwable;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..c5579d9
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeBatchParallelReadMergeSorterImpl.java
@@ -0,0 +1,338 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.BlockingQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+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.memory.MemoryException;
+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.row.CarbonSortBatch;
+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.exception.CarbonSortKeyAndGroupByException;
+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.
+ * It sorts data in batches and send to the next step.
+ */
+public class UnsafeBatchParallelReadMergeSorterImpl extends AbstractMergeSorter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeBatchParallelReadMergeSorterImpl.class.getName());
+
+  private SortParameters sortParameters;
+
+  private ExecutorService executorService;
+
+  private AtomicLong rowCounter;
+
+  public UnsafeBatchParallelReadMergeSorterImpl(AtomicLong rowCounter) {
+    this.rowCounter = rowCounter;
+  }
+
+  @Override public void initialize(SortParameters sortParameters) {
+    this.sortParameters = sortParameters;
+
+  }
+
+  @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
+      throws CarbonDataLoadingException {
+    this.executorService = Executors.newFixedThreadPool(iterators.length);
+    this.threadStatusObserver = new ThreadStatusObserver(this.executorService);
+    int batchSize = CarbonProperties.getInstance().getBatchSize();
+    final SortBatchHolder sortBatchHolder = new SortBatchHolder(sortParameters, iterators.length,
+        this.threadStatusObserver);
+
+    try {
+      for (int i = 0; i < iterators.length; i++) {
+        executorService.execute(
+            new SortIteratorThread(iterators[i], sortBatchHolder, batchSize, rowCounter,
+                this.threadStatusObserver));
+      }
+    } catch (Exception e) {
+      checkError();
+      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
+    }
+    checkError();
+    // Creates the iterator to read from merge sorter.
+    Iterator<CarbonSortBatch> batchIterator = new CarbonIterator<CarbonSortBatch>() {
+
+      @Override public boolean hasNext() {
+        return sortBatchHolder.hasNext();
+      }
+
+      @Override public CarbonSortBatch next() {
+        return new CarbonSortBatch(sortBatchHolder.next());
+      }
+    };
+    return new Iterator[] { batchIterator };
+  }
+
+  @Override public void close() {
+    executorService.shutdown();
+    try {
+      executorService.awaitTermination(2, TimeUnit.DAYS);
+    } catch (InterruptedException e) {
+      LOGGER.error(e);
+    }
+  }
+
+  /**
+   * This thread iterates the iterator and adds the rows
+   */
+  private static class SortIteratorThread implements Runnable {
+
+    private Iterator<CarbonRowBatch> iterator;
+
+    private SortBatchHolder sortDataRows;
+
+    private Object[][] buffer;
+
+    private AtomicLong rowCounter;
+
+    private ThreadStatusObserver threadStatusObserver;
+
+    public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortBatchHolder sortDataRows,
+        int batchSize, AtomicLong rowCounter, ThreadStatusObserver threadStatusObserver) {
+      this.iterator = iterator;
+      this.sortDataRows = sortDataRows;
+      this.buffer = new Object[batchSize][];
+      this.rowCounter = rowCounter;
+      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) {
+              buffer[i++] = row.getData();
+            }
+          }
+          if (i > 0) {
+            synchronized (sortDataRows) {
+              sortDataRows.getSortDataRow().addRowBatchWithOutSync(buffer, i);
+              rowCounter.getAndAdd(i);
+              if (!sortDataRows.getSortDataRow().canAdd()) {
+                sortDataRows.finish(false);
+                sortDataRows.createSortDataRows();
+              }
+            }
+          }
+        }
+      } catch (Exception e) {
+        LOGGER.error(e);
+        this.threadStatusObserver.notifyFailed(e);
+      } finally {
+        sortDataRows.finishThread();
+      }
+    }
+
+  }
+
+  private static class SortBatchHolder
+      extends CarbonIterator<UnsafeSingleThreadFinalSortFilesMerger> {
+
+    private SortParameters sortParameters;
+
+    private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
+
+    private UnsafeIntermediateMerger unsafeIntermediateFileMerger;
+
+    private UnsafeSortDataRows sortDataRow;
+
+    private final BlockingQueue<UnsafeSingleThreadFinalSortFilesMerger> mergerQueue;
+
+    private AtomicInteger iteratorCount;
+
+    private int batchCount;
+
+    private ThreadStatusObserver threadStatusObserver;
+
+    private final Object lock = new Object();
+
+    public SortBatchHolder(SortParameters sortParameters, int numberOfThreads,
+        ThreadStatusObserver threadStatusObserver) {
+      this.sortParameters = sortParameters.getCopy();
+      this.iteratorCount = new AtomicInteger(numberOfThreads);
+      this.mergerQueue = new LinkedBlockingQueue<>(1);
+      this.threadStatusObserver = threadStatusObserver;
+      createSortDataRows();
+    }
+
+    private void createSortDataRows() {
+      int inMemoryChunkSizeInMB = CarbonProperties.getInstance().getSortMemoryChunkSizeInMB();
+      setTempLocation(sortParameters);
+      this.finalMerger = new UnsafeSingleThreadFinalSortFilesMerger(sortParameters,
+          sortParameters.getTempFileLocation());
+      unsafeIntermediateFileMerger = new UnsafeIntermediateMerger(sortParameters);
+      sortDataRow = new UnsafeSortDataRows(sortParameters, unsafeIntermediateFileMerger,
+          inMemoryChunkSizeInMB);
+
+      try {
+        sortDataRow.initialize();
+      } catch (MemoryException e) {
+        throw new CarbonDataLoadingException(e);
+      }
+      batchCount++;
+    }
+
+    private void setTempLocation(SortParameters parameters) {
+      String[] carbonDataDirectoryPath = CarbonDataProcessorUtil
+          .getLocalDataFolderLocation(parameters.getDatabaseName(),
+            parameters.getTableName(), parameters.getTaskNo(), batchCount + "",
+            parameters.getSegmentId(), false, false);
+      String[] tempDirs = CarbonDataProcessorUtil.arrayAppend(carbonDataDirectoryPath,
+          File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
+      parameters.setTempFileLocation(tempDirs);
+    }
+
+    @Override public UnsafeSingleThreadFinalSortFilesMerger next() {
+      try {
+        UnsafeSingleThreadFinalSortFilesMerger unsafeSingleThreadFinalSortFilesMerger =
+            mergerQueue.take();
+        if (unsafeSingleThreadFinalSortFilesMerger.isStopProcess()) {
+          throw new RuntimeException(threadStatusObserver.getThrowable());
+        }
+        return unsafeSingleThreadFinalSortFilesMerger;
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    public UnsafeSortDataRows getSortDataRow() {
+      return sortDataRow;
+    }
+
+    public void finish(boolean isFinalAttempt) {
+      try {
+        // if the mergerQue is empty and some CarbonDataLoadingException exception has occurred
+        // then set stop process to true in the finalmerger instance
+        if (mergerQueue.isEmpty() && threadStatusObserver != null
+            && threadStatusObserver.getThrowable() != null && threadStatusObserver
+            .getThrowable() instanceof CarbonDataLoadingException) {
+          finalMerger.setStopProcess(true);
+          if (isFinalAttempt) {
+            iteratorCount.decrementAndGet();
+          }
+          mergerQueue.put(finalMerger);
+          return;
+        }
+        processRowToNextStep(sortDataRow, sortParameters);
+        unsafeIntermediateFileMerger.finish();
+        List<UnsafeCarbonRowPage> rowPages = unsafeIntermediateFileMerger.getRowPages();
+        finalMerger.startFinalMerge(rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]),
+            unsafeIntermediateFileMerger.getMergedPages());
+        unsafeIntermediateFileMerger.close();
+        if (isFinalAttempt) {
+          iteratorCount.decrementAndGet();
+        }
+        mergerQueue.put(finalMerger);
+        sortDataRow = null;
+        unsafeIntermediateFileMerger = null;
+        finalMerger = null;
+      } catch (CarbonDataWriterException e) {
+        throw new CarbonDataLoadingException(e);
+      } catch (CarbonSortKeyAndGroupByException e) {
+        throw new CarbonDataLoadingException(e);
+      } catch (InterruptedException e) {
+        // if fails to put in queue because of interrupted exception, we can offer to free the main
+        // thread from waiting.
+        if (finalMerger != null) {
+          finalMerger.setStopProcess(true);
+          boolean offered = mergerQueue.offer(finalMerger);
+          if (!offered) {
+            throw new CarbonDataLoadingException(e);
+          }
+        }
+        throw new CarbonDataLoadingException(e);
+      }
+    }
+
+    public void finishThread() {
+      synchronized (lock) {
+        if (iteratorCount.get() <= 1) {
+          finish(true);
+        } else {
+          iteratorCount.decrementAndGet();
+        }
+      }
+    }
+
+    public boolean hasNext() {
+      return iteratorCount.get() > 0 || !mergerQueue.isEmpty();
+    }
+
+    /**
+     * Below method will be used to process data to next step
+     */
+    private boolean processRowToNextStep(UnsafeSortDataRows sortDataRows, SortParameters parameters)
+        throws CarbonDataLoadingException {
+      if (null == sortDataRows) {
+        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 {
+        // start sorting
+        sortDataRows.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 (InterruptedException e) {
+        throw new CarbonDataLoadingException(e);
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java
new file mode 100644
index 0000000..1a2f704
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.sort.impl;
+
+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.datastore.exception.CarbonDataWriterException;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.memory.MemoryException;
+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.exception.CarbonSortKeyAndGroupByException;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+
+/**
+ * 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.
+ */
+public class UnsafeParallelReadMergeSorterImpl extends AbstractMergeSorter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeParallelReadMergeSorterImpl.class.getName());
+
+  private SortParameters sortParameters;
+
+  private UnsafeIntermediateMerger unsafeIntermediateFileMerger;
+
+  private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
+
+  private AtomicLong rowCounter;
+
+  public UnsafeParallelReadMergeSorterImpl(AtomicLong rowCounter) {
+    this.rowCounter = rowCounter;
+  }
+
+  @Override public void initialize(SortParameters sortParameters) {
+    this.sortParameters = sortParameters;
+    unsafeIntermediateFileMerger = new UnsafeIntermediateMerger(sortParameters);
+
+    finalMerger = new UnsafeSingleThreadFinalSortFilesMerger(sortParameters,
+        sortParameters.getTempFileLocation());
+  }
+
+  @Override public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
+      throws CarbonDataLoadingException {
+    int inMemoryChunkSizeInMB = CarbonProperties.getInstance().getSortMemoryChunkSizeInMB();
+    UnsafeSortDataRows sortDataRow =
+        new UnsafeSortDataRows(sortParameters, unsafeIntermediateFileMerger, inMemoryChunkSizeInMB);
+    final int batchSize = CarbonProperties.getInstance().getBatchSize();
+    try {
+      sortDataRow.initialize();
+    } catch (MemoryException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
+    this.threadStatusObserver = new ThreadStatusObserver(executorService);
+
+    try {
+      for (int i = 0; i < iterators.length; i++) {
+        executorService.execute(
+            new SortIteratorThread(iterators[i], sortDataRow, batchSize, rowCounter,
+                this.threadStatusObserver));
+      }
+      executorService.shutdown();
+      executorService.awaitTermination(2, TimeUnit.DAYS);
+      processRowToNextStep(sortDataRow, sortParameters);
+    } catch (Exception e) {
+      checkError();
+      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
+    }
+    checkError();
+    try {
+      unsafeIntermediateFileMerger.finish();
+      List<UnsafeCarbonRowPage> rowPages = unsafeIntermediateFileMerger.getRowPages();
+      finalMerger.startFinalMerge(rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]),
+          unsafeIntermediateFileMerger.getMergedPages());
+    } catch (CarbonDataWriterException e) {
+      throw new CarbonDataLoadingException(e);
+    } catch (CarbonSortKeyAndGroupByException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+
+    // Creates the iterator to read from merge sorter.
+    Iterator<CarbonRowBatch> batchIterator = new CarbonIterator<CarbonRowBatch>() {
+
+      @Override public boolean hasNext() {
+        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;
+      }
+    };
+    return new Iterator[] { batchIterator };
+  }
+
+  @Override public void close() {
+    unsafeIntermediateFileMerger.close();
+    finalMerger.clear();
+  }
+
+  /**
+   * Below method will be used to process data to next step
+   */
+  private boolean processRowToNextStep(UnsafeSortDataRows sortDataRows, SortParameters parameters)
+      throws CarbonDataLoadingException {
+    if (null == sortDataRows) {
+      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 {
+      // start sorting
+      sortDataRows.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 (InterruptedException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+  }
+
+  /**
+   * This thread iterates the iterator and adds the rows
+   */
+  private static class SortIteratorThread implements Runnable {
+
+    private Iterator<CarbonRowBatch> iterator;
+
+    private UnsafeSortDataRows sortDataRows;
+
+    private Object[][] buffer;
+
+    private AtomicLong rowCounter;
+
+    private ThreadStatusObserver threadStatusObserver;
+
+    public SortIteratorThread(Iterator<CarbonRowBatch> iterator,
+        UnsafeSortDataRows sortDataRows, int batchSize, AtomicLong rowCounter,
+        ThreadStatusObserver threadStatusObserver) {
+      this.iterator = iterator;
+      this.sortDataRows = sortDataRows;
+      this.buffer = new Object[batchSize][];
+      this.rowCounter = rowCounter;
+      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) {
+              buffer[i++] = row.getData();
+            }
+          }
+          if (i > 0) {
+            sortDataRows.addRowBatch(buffer, i);
+            rowCounter.getAndAdd(i);
+          }
+        }
+      } catch (Exception e) {
+        LOGGER.error(e);
+        this.threadStatusObserver.notifyFailed(e);
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..3c48e4d
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterWithBucketingImpl.java
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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(String.valueOf(i), 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);
+    // 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.getPartitionID(), 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 String partitionId;
+
+    private int batchSize;
+
+    private boolean firstRow;
+
+    private UnsafeIntermediateMerger intermediateMerger;
+
+    public MergedDataIterator(String partitionId, int batchSize,
+        UnsafeIntermediateMerger intermediateMerger) {
+      this.partitionId = partitionId;
+      this.batchSize = batchSize;
+      this.intermediateMerger = intermediateMerger;
+      this.firstRow = true;
+    }
+
+    private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
+
+    @Override public boolean hasNext() {
+      if (firstRow) {
+        firstRow = false;
+        finalMerger = getFinalMerger(partitionId);
+        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;
+    }
+  }
+}


[13/50] [abbrv] carbondata git commit: [CARBONDATA-1278] Data Mismatch issue when dictionary column filter values doesn't exists in dictionary

Posted by ja...@apache.org.
[CARBONDATA-1278] Data Mismatch issue when dictionary column filter values doesn't exists in dictionary

Root cause: when filter value is not present in dictionary, end key for that column is updated with 0,and hence btree jump is not selecting all the leaf node.
Fix:
To handle this issue end key should be updated with Integer max value, so that it can go till last leafnode of btree

This closes #1383


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

Branch: refs/heads/streaming_ingest
Commit: d5dd78afd237f10593c4403d9bf1d3bc482d3636
Parents: 6627cac
Author: kumarvishal <ku...@gmail.com>
Authored: Mon Sep 25 18:15:20 2017 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Wed Sep 27 14:12:00 2017 +0530

----------------------------------------------------------------------
 .../apache/carbondata/core/scan/filter/FilterUtil.java  | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/d5dd78af/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 497ca8c..e8488f9 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
@@ -1066,12 +1066,16 @@ public final class FilterUtil {
         continue;
       }
       int keyOrdinalOfDimensionFromCurrentBlock = dimensionFromCurrentBlock.getKeyOrdinal();
+      int endFilterValue = 0;
       for (ColumnFilterInfo info : values) {
         if (keyOrdinalOfDimensionFromCurrentBlock < endKey.length) {
-          if (endKey[keyOrdinalOfDimensionFromCurrentBlock] > info.getFilterList()
-              .get(info.getFilterList().size() - 1)) {
-            endKey[keyOrdinalOfDimensionFromCurrentBlock] =
-                info.getFilterList().get(info.getFilterList().size() - 1);
+          endFilterValue = info.getFilterList().get(info.getFilterList().size() - 1);
+          if (endFilterValue == 0) {
+            endFilterValue =
+                segmentProperties.getDimColumnsCardinality()[keyOrdinalOfDimensionFromCurrentBlock];
+          }
+          if (endKey[keyOrdinalOfDimensionFromCurrentBlock] > endFilterValue) {
+            endKey[keyOrdinalOfDimensionFromCurrentBlock] = endFilterValue;
           }
         }
       }


[10/50] [abbrv] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
deleted file mode 100644
index 7ed280e..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ /dev/null
@@ -1,1315 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command
-
-import java.text.SimpleDateFormat
-import java.util
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
-import scala.language.implicitConversions
-
-import org.apache.commons.lang3.StringUtils
-import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
-import org.apache.spark.sql.catalyst.expressions.Attribute
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
-import org.apache.spark.util.{AlterTableUtil, CausedBy, FileUtils, PartitionUtils}
-import org.codehaus.jackson.map.ObjectMapper
-
-import org.apache.carbondata.api.CarbonStore
-import org.apache.carbondata.common.constants.LoggerAction
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.cache.CacheProvider
-import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonLoadOptionConstants}
-import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.dictionary.server.DictionaryServer
-import org.apache.carbondata.core.exception.InvalidConfigurationException
-import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.PartitionInfo
-import org.apache.carbondata.core.metadata.schema.partition.PartitionType
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
-import org.apache.carbondata.core.metadata.schema.table.TableInfo
-import org.apache.carbondata.core.mutate.{CarbonUpdateUtil, TupleIdEnum}
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
-import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.format
-import org.apache.carbondata.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.etl.DataLoadingException
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.newflow.exception.NoRetryException
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.load.ValidateUtil
-import org.apache.carbondata.spark.rdd.{CarbonDataRDDFactory, DictionaryLoadModel}
-import org.apache.carbondata.spark.util.{CarbonSparkUtil, CommonUtil, GlobalDictionaryUtil}
-
-object Checker {
-  def validateTableExists(
-      dbName: Option[String],
-      tableName: String,
-      session: SparkSession): Unit = {
-    val identifier = TableIdentifier(tableName, dbName)
-    if (!CarbonEnv.getInstance(session).carbonMetastore.tableExists(identifier)(session)) {
-      val err = s"table $dbName.$tableName not found"
-      LogServiceFactory.getLogService(this.getClass.getName).error(err)
-      throw new IllegalArgumentException(err)
-    }
-  }
-}
-
-/**
- * Interface for command that modifies schema
- */
-trait SchemaProcessCommand {
-  def processSchema(sparkSession: SparkSession): Seq[Row]
-}
-
-/**
- * Interface for command that need to process data in file system
- */
-trait DataProcessCommand {
-  def processData(sparkSession: SparkSession): Seq[Row]
-}
-
-/**
- * Command for show table partitions Command
- *
- * @param tableIdentifier
- */
-private[sql] case class ShowCarbonPartitionsCommand(
-    tableIdentifier: TableIdentifier) extends RunnableCommand with SchemaProcessCommand {
-
-  override val output = CommonUtil.partitionInfoOutput
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(tableIdentifier)(sparkSession).
-        asInstanceOf[CarbonRelation]
-    val carbonTable = relation.tableMeta.carbonTable
-    val tableName = carbonTable.getFactTableName
-    val partitionInfo = carbonTable.getPartitionInfo(
-      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
-    if (partitionInfo == null) {
-      throw new AnalysisException(
-        s"SHOW PARTITIONS is not allowed on a table that is not partitioned: $tableName")
-    }
-    val partitionType = partitionInfo.getPartitionType
-    val columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
-    val LOGGER = LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
-    LOGGER.info("partition column name:" + columnName)
-    CommonUtil.getPartitionInfo(columnName, partitionType, partitionInfo)
-  }
-}
-
-/**
- * Command for the compaction in alter table command
- *
- * @param alterTableModel
- */
-case class AlterTableCompaction(alterTableModel: AlterTableModel) extends RunnableCommand
-    with DataProcessCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableModel.tableName.toLowerCase
-    val databaseName = alterTableModel.dbName.getOrElse(sparkSession.catalog.currentDatabase)
-    val relation =
-      CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .lookupRelation(Option(databaseName), tableName)(sparkSession)
-          .asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      sys.error(s"Table $databaseName.$tableName does not exist")
-    }
-    if (null == relation.tableMeta.carbonTable) {
-      LOGGER.error(s"alter table failed. table not found: $databaseName.$tableName")
-      sys.error(s"alter table failed. table not found: $databaseName.$tableName")
-    }
-
-    val carbonLoadModel = new CarbonLoadModel()
-
-    val table = relation.tableMeta.carbonTable
-    carbonLoadModel.setTableName(table.getFactTableName)
-    val dataLoadSchema = new CarbonDataLoadSchema(table)
-    // Need to fill dimension relation
-    carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-    carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
-    carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
-    carbonLoadModel.setStorePath(relation.tableMeta.carbonTable.getStorePath)
-
-    var storeLocation = CarbonProperties.getInstance
-        .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
-          System.getProperty("java.io.tmpdir")
-        )
-    storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
-    try {
-      CarbonDataRDDFactory
-          .alterTableForCompaction(sparkSession.sqlContext,
-            alterTableModel,
-            carbonLoadModel,
-            storeLocation
-          )
-    } catch {
-      case e: Exception =>
-        if (null != e.getMessage) {
-          sys.error(s"Compaction failed. Please check logs for more info. ${ e.getMessage }")
-        } else {
-          sys.error("Exception in compaction. Please check logs for more info.")
-        }
-    }
-    Seq.empty
-  }
-}
-
-/**
- * Command for Alter Table Add & Split partition
- * Add is a special case of Splitting the default partition (part0)
- * @param splitPartitionModel
- */
-case class AlterTableSplitPartitionCommand(splitPartitionModel: AlterTableSplitPartitionModel)
-  extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-  val tableName = splitPartitionModel.tableName
-  val splitInfo = splitPartitionModel.splitInfo
-  val partitionId = splitPartitionModel.partitionId.toInt
-  var partitionInfo: PartitionInfo = null
-  var carbonMetaStore: CarbonMetaStore = null
-  var relation: CarbonRelation = null
-  var dbName: String = null
-  var storePath: String = null
-  var table: CarbonTable = null
-  var carbonTableIdentifier: CarbonTableIdentifier = null
-  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
-  val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
-    .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-      CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
-  val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
-    .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
-      CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
-  val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-    LockUsage.COMPACTION_LOCK,
-    LockUsage.DELETE_SEGMENT_LOCK,
-    LockUsage.DROP_TABLE_LOCK,
-    LockUsage.CLEAN_FILES_LOCK,
-    LockUsage.ALTER_PARTITION_LOCK)
-
-  // TODO will add rollback function incase process data failure
-  def run(sparkSession: SparkSession): Seq[Row] = {
-      processSchema(sparkSession)
-      processData(sparkSession)
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    dbName = splitPartitionModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
-      .asInstanceOf[CarbonRelation]
-    carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
-    storePath = relation.tableMeta.storePath
-    if (relation == null) {
-      sys.error(s"Table $dbName.$tableName does not exist")
-    }
-    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
-    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
-      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
-      sys.error(s"Alter table failed. table not found: $dbName.$tableName")
-    }
-    table = relation.tableMeta.carbonTable
-    partitionInfo = table.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
-    oldPartitionIds.addAll(partitionIds.asJava)
-
-    if (partitionInfo == null) {
-      sys.error(s"Table $tableName is not a partition table.")
-    }
-    if (partitionInfo.getPartitionType == PartitionType.HASH) {
-      sys.error(s"Hash partition table cannot be added or split!")
-    }
-    PartitionUtils.updatePartitionInfo(partitionInfo, partitionIds, partitionId,
-      splitInfo, timestampFormatter, dateFormatter)
-
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
-    val schemaFilePath = carbonTablePath.getSchemaFilePath
-    // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
-    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
-      dbName, tableName, storePath)
-    val tableSchema = wrapperTableInfo.getFactTable
-    tableSchema.setPartitionInfo(partitionInfo)
-    wrapperTableInfo.setFactTable(tableSchema)
-    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
-    val thriftTable =
-      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-    carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
-      dbName, tableName, storePath)
-    CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
-    // update the schema modified time
-    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
-    sparkSession.catalog.refreshTable(tableName)
-    Seq.empty
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    var locks = List.empty[ICarbonLock]
-    var success = false
-    try {
-      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
-        locksToBeAcquired)(sparkSession)
-      val carbonLoadModel = new CarbonLoadModel()
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-      carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
-      carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
-      carbonLoadModel.setStorePath(storePath)
-      val loadStartTime = CarbonUpdateUtil.readCurrentTime
-      carbonLoadModel.setFactTimeStamp(loadStartTime)
-      CarbonDataRDDFactory.alterTableSplitPartition(sparkSession.sqlContext,
-        partitionId.toString,
-        carbonLoadModel,
-        oldPartitionIds.asScala.toList
-      )
-      success = true
-    } catch {
-      case e: Exception =>
-        success = false
-        sys.error(s"Add/Split Partition failed. Please check logs for more info. ${ e.getMessage }")
-    } finally {
-      AlterTableUtil.releaseLocks(locks)
-      CacheProvider.getInstance().dropAllCache()
-      LOGGER.info("Locks released after alter table add/split partition action.")
-      LOGGER.audit("Locks released after alter table add/split partition action.")
-      if (success) {
-        LOGGER.info(s"Alter table add/split partition is successful for table $dbName.$tableName")
-        LOGGER.audit(s"Alter table add/split partition is successful for table $dbName.$tableName")
-      }
-    }
-    Seq.empty
-  }
-}
-
-case class AlterTableDropPartition(alterTableDropPartitionModel: AlterTableDropPartitionModel)
-  extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-  val tableName = alterTableDropPartitionModel.tableName
-  var dbName: String = null
-  val partitionId = alterTableDropPartitionModel.partitionId
-  val dropWithData = alterTableDropPartitionModel.dropWithData
-  if (partitionId == 0 ) {
-    sys.error(s"Cannot drop default partition! Please use delete statement!")
-  }
-  var partitionInfo: PartitionInfo = null
-  var carbonMetaStore: CarbonMetaStore = null
-  var relation: CarbonRelation = null
-  var storePath: String = null
-  var table: CarbonTable = null
-  var carbonTableIdentifier: CarbonTableIdentifier = null
-  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
-  val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-    LockUsage.COMPACTION_LOCK,
-    LockUsage.DELETE_SEGMENT_LOCK,
-    LockUsage.DROP_TABLE_LOCK,
-    LockUsage.CLEAN_FILES_LOCK,
-    LockUsage.ALTER_PARTITION_LOCK)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-    processData(sparkSession)
-    Seq.empty
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    dbName = alterTableDropPartitionModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
-      .asInstanceOf[CarbonRelation]
-    carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
-    storePath = relation.tableMeta.storePath
-    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
-    if (relation == null) {
-      sys.error(s"Table $dbName.$tableName does not exist")
-    }
-    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
-      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
-      sys.error(s"Alter table failed. table not found: $dbName.$tableName")
-    }
-    table = relation.tableMeta.carbonTable
-    partitionInfo = table.getPartitionInfo(tableName)
-    if (partitionInfo == null) {
-      sys.error(s"Table $tableName is not a partition table.")
-    }
-    val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
-    // keep a copy of partitionIdList before update partitionInfo.
-    // will be used in partition data scan
-    oldPartitionIds.addAll(partitionIds.asJava)
-    val partitionIndex = partitionIds.indexOf(Integer.valueOf(partitionId))
-    partitionInfo.getPartitionType match {
-      case PartitionType.HASH => sys.error(s"Hash partition cannot be dropped!")
-      case PartitionType.RANGE =>
-        val rangeInfo = new util.ArrayList(partitionInfo.getRangeInfo)
-        val rangeToRemove = partitionInfo.getRangeInfo.get(partitionIndex - 1)
-        rangeInfo.remove(rangeToRemove)
-        partitionInfo.setRangeInfo(rangeInfo)
-      case PartitionType.LIST =>
-        val listInfo = new util.ArrayList(partitionInfo.getListInfo)
-        val listToRemove = partitionInfo.getListInfo.get(partitionIndex - 1)
-        listInfo.remove(listToRemove)
-        partitionInfo.setListInfo(listInfo)
-      case PartitionType.RANGE_INTERVAL =>
-        sys.error(s"Dropping range interval partition isn't support yet!")
-    }
-    partitionInfo.dropPartition(partitionIndex)
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
-    val schemaFilePath = carbonTablePath.getSchemaFilePath
-    // read TableInfo
-    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
-
-    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
-      dbName, tableName, storePath)
-    val tableSchema = wrapperTableInfo.getFactTable
-    tableSchema.setPartitionInfo(partitionInfo)
-    wrapperTableInfo.setFactTable(tableSchema)
-    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
-    val thriftTable =
-      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-    thriftTable.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-      .setTime_stamp(System.currentTimeMillis)
-    carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
-      dbName, tableName, storePath)
-    CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
-    // update the schema modified time
-    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
-    // sparkSession.catalog.refreshTable(tableName)
-    Seq.empty
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    var locks = List.empty[ICarbonLock]
-    var success = false
-    try {
-      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
-        locksToBeAcquired)(sparkSession)
-      val carbonLoadModel = new CarbonLoadModel()
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      // Need to fill dimension relation
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-      carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
-      carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
-      carbonLoadModel.setStorePath(storePath)
-      val loadStartTime = CarbonUpdateUtil.readCurrentTime
-      carbonLoadModel.setFactTimeStamp(loadStartTime)
-      CarbonDataRDDFactory.alterTableDropPartition(sparkSession.sqlContext,
-        partitionId,
-        carbonLoadModel,
-        dropWithData,
-        oldPartitionIds.asScala.toList
-      )
-      success = true
-    } catch {
-      case e: Exception =>
-        sys.error(s"Drop Partition failed. Please check logs for more info. ${ e.getMessage } ")
-      success = false
-    } finally {
-      CacheProvider.getInstance().dropAllCache()
-      AlterTableUtil.releaseLocks(locks)
-      LOGGER.info("Locks released after alter table drop partition action.")
-      LOGGER.audit("Locks released after alter table drop partition action.")
-    }
-    LOGGER.info(s"Alter table drop partition is successful for table $dbName.$tableName")
-    LOGGER.audit(s"Alter table drop partition is successful for table $dbName.$tableName")
-    Seq.empty
-  }
-}
-
-  case class CreateTable(cm: TableModel, createDSTable: Boolean = true) extends RunnableCommand
-    with SchemaProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-  }
-
-  def setV(ref: Any, name: String, value: Any): Unit = {
-    ref.getClass.getFields.find(_.getName == name).get
-      .set(ref, value.asInstanceOf[AnyRef])
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val storePath = CarbonEnv.getInstance(sparkSession).storePath
-    CarbonEnv.getInstance(sparkSession).carbonMetastore.
-      checkSchemasModifiedTimeAndReloadTables(storePath)
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    cm.databaseName = getDB.getDatabaseName(cm.databaseNameOp, sparkSession)
-    val tbName = cm.tableName
-    val dbName = cm.databaseName
-    LOGGER.audit(s"Creating Table with Database name [$dbName] and Table name [$tbName]")
-
-    val tableInfo: TableInfo = TableNewProcessor(cm)
-
-    // Add validation for sort scope when create table
-    val sortScope = tableInfo.getFactTable.getTableProperties
-      .getOrDefault("sort_scope", CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT)
-    if (!CarbonUtil.isValidSortOption(sortScope)) {
-      throw new InvalidConfigurationException(s"Passing invalid SORT_SCOPE '$sortScope'," +
-        s" valid SORT_SCOPE are 'NO_SORT', 'BATCH_SORT', 'LOCAL_SORT' and 'GLOBAL_SORT' ")
-    }
-
-    if (tableInfo.getFactTable.getListOfColumns.size <= 0) {
-      sys.error("No Dimensions found. Table should have at least one dimesnion !")
-    }
-
-    if (sparkSession.sessionState.catalog.listTables(dbName)
-        .exists(_.table.equalsIgnoreCase(tbName))) {
-      if (!cm.ifNotExistsSet) {
-        LOGGER.audit(
-          s"Table creation with Database name [$dbName] and Table name [$tbName] failed. " +
-              s"Table [$tbName] already exists under database [$dbName]")
-        sys.error(s"Table [$tbName] already exists under database [$dbName]")
-      }
-    } else {
-      val tableIdentifier = AbsoluteTableIdentifier.from(storePath, dbName, tbName)
-      // Add Database to catalog and persist
-      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      val tablePath = tableIdentifier.getTablePath
-      val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tablePath)
-      if (createDSTable) {
-        try {
-          val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
-          cm.dimCols.foreach(f => fields(f.schemaOrdinal) = f)
-          cm.msrCols.foreach(f => fields(f.schemaOrdinal) = f)
-
-          sparkSession.sql(
-            s"""CREATE TABLE $dbName.$tbName
-               |(${ fields.map(f => f.rawSchema).mkString(",") })
-               |USING org.apache.spark.sql.CarbonSource""".stripMargin +
-                s""" OPTIONS (tableName "$tbName", dbName "$dbName", tablePath """.stripMargin +
-                s""""$tablePath"$carbonSchemaString) """)
-        } catch {
-          case e: Exception =>
-            val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
-            // call the drop table to delete the created table.
-            CarbonEnv.getInstance(sparkSession).carbonMetastore
-                .dropTable(tablePath, identifier)(sparkSession)
-
-            LOGGER.audit(s"Table creation with Database name [$dbName] " +
-                s"and Table name [$tbName] failed")
-            throw e
-        }
-      }
-
-      LOGGER.audit(s"Table created with Database name [$dbName] and Table name [$tbName]")
-    }
-    Seq.empty
-  }
-}
-
-case class DeleteLoadsById(
-    loadids: Seq[String],
-    databaseNameOp: Option[String],
-    tableName: String) extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
-    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
-        lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
-        tableMeta.carbonTable
-    CarbonStore.deleteLoadById(
-      loadids,
-      getDB.getDatabaseName(databaseNameOp, sparkSession),
-      tableName,
-      carbonTable
-    )
-    Seq.empty
-  }
-}
-
-case class DeleteLoadsByLoadDate(
-    databaseNameOp: Option[String],
-    tableName: String,
-    dateField: String,
-    loadDate: String) extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
-    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
-        lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
-        tableMeta.carbonTable
-    CarbonStore.deleteLoadByDate(
-      loadDate,
-      getDB.getDatabaseName(databaseNameOp, sparkSession),
-      tableName,
-      carbonTable
-    )
-    Seq.empty
-  }
-}
-
-object LoadTable {
-
-  def updateTableMetadata(carbonLoadModel: CarbonLoadModel,
-      sqlContext: SQLContext,
-      model: DictionaryLoadModel,
-      noDictDimension: Array[CarbonDimension]): Unit = {
-    val sparkSession = sqlContext.sparkSession
-    val carbonTablePath = CarbonStorePath.getCarbonTablePath(model.hdfsLocation,
-      model.table)
-
-    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    // read TableInfo
-    val tableInfo: format.TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-
-    // modify TableInfo
-    val columns = tableInfo.getFact_table.getTable_columns
-    for (i <- 0 until columns.size) {
-      if (noDictDimension.exists(x => columns.get(i).getColumn_id.equals(x.getColumnId))) {
-        columns.get(i).encoders.remove(org.apache.carbondata.format.Encoding.DICTIONARY)
-      }
-    }
-    val entry = tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-    entry.setTime_stamp(System.currentTimeMillis())
-
-    // write TableInfo
-    metastore.updateTableSchema(carbonTablePath.getCarbonTableIdentifier,
-      carbonTablePath.getCarbonTableIdentifier,
-      tableInfo, entry, carbonTablePath.getPath)(sparkSession)
-
-    // update the schema modified time
-    metastore.updateAndTouchSchemasUpdatedTime(model.hdfsLocation)
-
-    // update CarbonDataLoadSchema
-    val carbonTable = metastore.lookupRelation(Option(model.table.getDatabaseName),
-      model.table.getTableName)(sqlContext.sparkSession).asInstanceOf[CarbonRelation].tableMeta
-      .carbonTable
-    carbonLoadModel.setCarbonDataLoadSchema(new CarbonDataLoadSchema(carbonTable))
-  }
-
-}
-
-case class LoadTableByInsert(relation: CarbonDatasourceHadoopRelation,
-    child: LogicalPlan,
-    overwrite: Boolean)
-  extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val df = Dataset.ofRows(sparkSession, child)
-    val header = relation.tableSchema.get.fields.map(_.name).mkString(",")
-    val load = LoadTable(
-      Some(relation.carbonRelation.databaseName),
-      relation.carbonRelation.tableName,
-      null,
-      Seq(),
-      scala.collection.immutable.Map("fileheader" -> header),
-      overwrite,
-      null,
-      Some(df)).run(sparkSession)
-    // updating relation metadata. This is in case of auto detect high cardinality
-    relation.carbonRelation.metaData =
-        CarbonSparkUtil.createSparkMeta(relation.carbonRelation.tableMeta.carbonTable)
-    load
-  }
-}
-
-case class LoadTable(
-    databaseNameOp: Option[String],
-    tableName: String,
-    factPathFromUser: String,
-    dimFilesPath: Seq[DataLoadTableFileMapping],
-    options: scala.collection.immutable.Map[String, String],
-    isOverwriteTable: Boolean,
-    var inputSqlString: String = null,
-    dataFrame: Option[DataFrame] = None,
-    updateModel: Option[UpdateTableModel] = None) extends RunnableCommand with DataProcessCommand {
-
-  private def getFinalOptions(carbonProperty: CarbonProperties):
-  scala.collection.mutable.Map[String, String] = {
-    var 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("escapechar", options.getOrElse("escapechar", "\\"))
-    optionsFinal.put("commentchar", options.getOrElse("commentchar", "#"))
-    optionsFinal.put("columndict", options.getOrElse("columndict", null))
-    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("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("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
-  }
-
-  private def checkDefaultValue(value: String, default: String) = {
-    if (StringUtils.isEmpty(value)) {
-      default
-    } else {
-      value
-    }
-  }
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    if (dataFrame.isDefined && updateModel.isEmpty) {
-      val rdd = dataFrame.get.rdd
-      if (rdd.partitions == null || rdd.partitions.length == 0) {
-        LOGGER.warn("DataLoading finished. No data was loaded.")
-        return Seq.empty
-      }
-    }
-
-    val dbName = databaseNameOp.getOrElse(sparkSession.catalog.currentDatabase)
-
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(Option(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      sys.error(s"Table $dbName.$tableName does not exist")
-    }
-    if (null == relation.tableMeta.carbonTable) {
-      LOGGER.error(s"Data loading failed. table not found: $dbName.$tableName")
-      LOGGER.audit(s"Data loading failed. table not found: $dbName.$tableName")
-      sys.error(s"Data loading failed. table not found: $dbName.$tableName")
-    }
-
-    val carbonProperty: CarbonProperties = CarbonProperties.getInstance()
-    carbonProperty.addProperty("zookeeper.enable.lock", "false")
-    val optionsFinal = getFinalOptions(carbonProperty)
-
-    val tableProperties = relation.tableMeta.carbonTable.getTableInfo
-      .getFactTable.getTableProperties
-
-    optionsFinal.put("sort_scope", tableProperties.getOrDefault("sort_scope",
-        carbonProperty.getProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_SORT_SCOPE,
-          carbonProperty.getProperty(CarbonCommonConstants.LOAD_SORT_SCOPE,
-            CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))))
-
-    try {
-      val factPath = if (dataFrame.isDefined) {
-        ""
-      } else {
-        FileUtils.getPaths(
-          CarbonUtil.checkAndAppendHDFSUrl(factPathFromUser))
-      }
-      val carbonLoadModel = new CarbonLoadModel()
-      carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
-      carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
-      carbonLoadModel.setStorePath(relation.tableMeta.carbonTable.getStorePath)
-
-      val table = relation.tableMeta.carbonTable
-      carbonLoadModel.setTableName(table.getFactTableName)
-      val dataLoadSchema = new CarbonDataLoadSchema(table)
-      // Need to fill dimension relation
-      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
-
-      val partitionLocation = relation.tableMeta.storePath + "/partition/" +
-          relation.tableMeta.carbonTableIdentifier.getDatabaseName + "/" +
-          relation.tableMeta.carbonTableIdentifier.getTableName + "/"
-      val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean
-      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")
-      val bad_record_path = optionsFinal("bad_record_path")
-      val global_sort_partitions = optionsFinal("global_sort_partitions")
-      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.validateDateFormat(dateFormat, table, tableName)
-      ValidateUtil.validateSortScope(table, sort_scope)
-
-      if (bad_records_logger_enable.toBoolean ||
-          LoggerAction.REDIRECT.name().equalsIgnoreCase(bad_records_action)) {
-        if (!CarbonUtil.isValidBadStorePath(bad_record_path)) {
-          sys.error("Invalid bad records location.")
-        }
-      }
-      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)
-        }
-        header match {
-          case true =>
-            if (fileHeader.nonEmpty) {
-              throw new MalformedCarbonCommandException(
-                "When 'header' option is true, 'fileheader' option is not required.")
-            }
-          case false =>
-            // generate file header
-            if (fileHeader.isEmpty) {
-              fileHeader = table.getCreateOrderColumn(table.getFactTableName)
-                  .asScala.map(_.getColName).mkString(",")
-            }
-        }
-      }
-
-      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)
-      carbonLoadModel
-          .setIsEmptyDataBadRecord(
-            DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," +
-                optionsFinal("is_empty_data_bad_record"))
-      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)) {
-        sys.error(s"Field Delimiter & 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)
-
-      val partitionStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
-      try {
-        // 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)")
-        carbonLoadModel.setFactFilePath(factPath)
-        carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimeter))
-        carbonLoadModel.setCsvHeader(fileHeader)
-        carbonLoadModel.setColDictFilePath(column_dict)
-        carbonLoadModel.setDirectLoad(true)
-        carbonLoadModel.setCsvHeaderColumns(CommonUtil.getCsvHeaderColumns(carbonLoadModel))
-        val validatedMaxColumns = CommonUtil.validateMaxColumns(carbonLoadModel.getCsvHeaderColumns,
-          optionsFinal("maxcolumns"))
-        carbonLoadModel.setMaxColumns(validatedMaxColumns.toString)
-        GlobalDictionaryUtil.updateTableMetadataFunc = LoadTable.updateTableMetadata
-        val storePath = relation.tableMeta.storePath
-        // add the start entry for the new load in the table status file
-        if (!updateModel.isDefined) {
-          CommonUtil.
-            readAndUpdateLoadProgressInTableMeta(carbonLoadModel, storePath, isOverwriteTable)
-        }
-        if (isOverwriteTable) {
-          LOGGER.info(s"Overwrite of carbon table with $dbName.$tableName is in progress")
-        }
-        if (null == carbonLoadModel.getLoadMetadataDetails) {
-          CommonUtil.readLoadMetadataDetails(carbonLoadModel)
-        }
-        if (carbonLoadModel.getLoadMetadataDetails.isEmpty && carbonLoadModel.getUseOnePass &&
-            StringUtils.isEmpty(column_dict) && StringUtils.isEmpty(all_dictionary_path)) {
-          LOGGER.info(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
-          LOGGER.audit(s"Cannot use single_pass=true for $dbName.$tableName during the first load")
-          carbonLoadModel.setUseOnePass(false)
-        }
-        // Create table and metadata folders if not exist
-        val carbonTablePath = CarbonStorePath
-            .getCarbonTablePath(storePath, table.getCarbonTableIdentifier)
-        val metadataDirectoryPath = carbonTablePath.getMetadataDirectoryPath
-        val fileType = FileFactory.getFileType(metadataDirectoryPath)
-        if (!FileFactory.isFileExist(metadataDirectoryPath, fileType)) {
-          FileFactory.mkdirs(metadataDirectoryPath, fileType)
-        }
-        if (carbonLoadModel.getUseOnePass) {
-          val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
-          val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-              .getCarbonTableIdentifier
-          val carbonTablePath = CarbonStorePath
-              .getCarbonTablePath(storePath, carbonTableIdentifier)
-          val dictFolderPath = carbonTablePath.getMetadataDirectoryPath
-          val dimensions = carbonTable.getDimensionByTableName(
-            carbonTable.getFactTableName).asScala.toArray
-          val colDictFilePath = carbonLoadModel.getColDictFilePath
-          if (!StringUtils.isEmpty(colDictFilePath)) {
-            carbonLoadModel.initPredefDictMap()
-            // generate predefined dictionary
-            GlobalDictionaryUtil
-                .generatePredefinedColDictionary(colDictFilePath, carbonTableIdentifier,
-                  dimensions, carbonLoadModel, sparkSession.sqlContext, storePath, dictFolderPath)
-          }
-          if (!StringUtils.isEmpty(all_dictionary_path)) {
-            carbonLoadModel.initPredefDictMap()
-            GlobalDictionaryUtil
-                .generateDictionaryFromDictionaryFiles(sparkSession.sqlContext,
-                  carbonLoadModel,
-                  storePath,
-                  carbonTableIdentifier,
-                  dictFolderPath,
-                  dimensions,
-                  all_dictionary_path)
-          }
-          // dictionaryServerClient dictionary generator
-          val dictionaryServerPort = carbonProperty
-              .getProperty(CarbonCommonConstants.DICTIONARY_SERVER_PORT,
-                CarbonCommonConstants.DICTIONARY_SERVER_PORT_DEFAULT)
-          val sparkDriverHost = sparkSession.sqlContext.sparkContext.
-              getConf.get("spark.driver.host")
-          carbonLoadModel.setDictionaryServerHost(sparkDriverHost)
-          // start dictionary server when use one pass load and dimension with DICTIONARY
-          // encoding is present.
-          val allDimensions = table.getAllDimensions.asScala.toList
-          val createDictionary = allDimensions.exists {
-            carbonDimension => carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
-                !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)
-          }
-          val server: Option[DictionaryServer] = if (createDictionary) {
-            val dictionaryServer = DictionaryServer
-              .getInstance(dictionaryServerPort.toInt, carbonTable)
-            carbonLoadModel.setDictionaryServerPort(dictionaryServer.getPort)
-            sparkSession.sparkContext.addSparkListener(new SparkListener() {
-              override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
-                dictionaryServer.shutdown()
-              }
-            })
-            Some(dictionaryServer)
-          } else {
-            None
-          }
-          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
-            carbonLoadModel,
-            relation.tableMeta.storePath,
-            columnar,
-            partitionStatus,
-            server,
-            isOverwriteTable,
-            dataFrame,
-            updateModel)
-        } else {
-          val (dictionaryDataFrame, loadDataFrame) = if (updateModel.isDefined) {
-            val fields = dataFrame.get.schema.fields
-            import org.apache.spark.sql.functions.udf
-            // extracting only segment from tupleId
-            val getSegIdUDF = udf((tupleId: String) =>
-              CarbonUpdateUtil.getRequiredFieldFromTID(tupleId, TupleIdEnum.SEGMENT_ID))
-            // getting all fields except tupleId field as it is not required in the value
-            var otherFields = fields.toSeq
-                .filter(field => !field.name
-                    .equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))
-                .map(field => new Column(field.name))
-
-            // extract tupleId field which will be used as a key
-            val segIdColumn = getSegIdUDF(new Column(UnresolvedAttribute
-                .quotedString(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID))).
-              as(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID)
-            // use dataFrameWithoutTupleId as dictionaryDataFrame
-            val dataFrameWithoutTupleId = dataFrame.get.select(otherFields: _*)
-            otherFields = otherFields :+ segIdColumn
-            // use dataFrameWithTupleId as loadDataFrame
-            val dataFrameWithTupleId = dataFrame.get.select(otherFields: _*)
-            (Some(dataFrameWithoutTupleId), Some(dataFrameWithTupleId))
-          } else {
-            (dataFrame, dataFrame)
-          }
-
-            GlobalDictionaryUtil.generateGlobalDictionary(
-                sparkSession.sqlContext,
-                carbonLoadModel,
-                relation.tableMeta.storePath,
-                dictionaryDataFrame)
-          CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
-            carbonLoadModel,
-            relation.tableMeta.storePath,
-            columnar,
-            partitionStatus,
-            None,
-            isOverwriteTable,
-            loadDataFrame,
-            updateModel)
-        }
-      } catch {
-        case CausedBy(ex: NoRetryException) =>
-          LOGGER.error(ex, s"Dataload failure for $dbName.$tableName")
-          throw new RuntimeException(s"Dataload failure for $dbName.$tableName, ${ex.getMessage}")
-        case ex: Exception =>
-          LOGGER.error(ex)
-          LOGGER.audit(s"Dataload failure for $dbName.$tableName. Please check the logs")
-          throw ex
-      } finally {
-        // Once the data load is successful delete the unwanted partition files
-        try {
-          val fileType = FileFactory.getFileType(partitionLocation)
-          if (FileFactory.isFileExist(partitionLocation, fileType)) {
-            val file = FileFactory
-                .getCarbonFile(partitionLocation, fileType)
-            CarbonUtil.deleteFoldersAndFiles(file)
-          }
-        } catch {
-          case ex: Exception =>
-            LOGGER.error(ex)
-            LOGGER.audit(s"Dataload failure for $dbName.$tableName. " +
-                "Problem deleting the partition folder")
-            throw ex
-        }
-
-      }
-    } catch {
-      case dle: DataLoadingException =>
-        LOGGER.audit(s"Dataload failed for $dbName.$tableName. " + dle.getMessage)
-        throw dle
-      case mce: MalformedCarbonCommandException =>
-        LOGGER.audit(s"Dataload failed for $dbName.$tableName. " + mce.getMessage)
-        throw mce
-    }
-    Seq.empty
-  }
-}
-
-case class CleanFiles(
-    databaseNameOp: Option[String],
-    tableName: String, forceTableClean: Boolean = false)
-  extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
-    if (forceTableClean) {
-      CarbonStore.cleanFiles(
-        getDB.getDatabaseName(databaseNameOp, sparkSession),
-        tableName,
-        CarbonEnv.getInstance(sparkSession).storePath,
-        null,
-        forceTableClean)
-    } else {
-      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      val relation = catalog
-        .lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation]
-      val carbonTable = relation.tableMeta.carbonTable
-      CarbonStore.cleanFiles(
-        getDB.getDatabaseName(databaseNameOp, sparkSession),
-        tableName,
-        relation.asInstanceOf[CarbonRelation].tableMeta.storePath,
-        carbonTable,
-        forceTableClean)
-    }
-    Seq.empty
-  }
-}
-
-case class ShowLoads(
-    databaseNameOp: Option[String],
-    tableName: String,
-    limit: Option[String],
-    override val output: Seq[Attribute]) extends RunnableCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processData(sparkSession)
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
-    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
-        lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
-        tableMeta.carbonTable
-    CarbonStore.showSegments(
-      getDB.getDatabaseName(databaseNameOp, sparkSession),
-      tableName,
-      limit,
-      carbonTable.getMetaDataFilepath
-    )
-  }
-}
-
-case class CarbonDropTableCommand(ifExistsSet: Boolean,
-    databaseNameOp: Option[String],
-    tableName: String)
-  extends RunnableCommand with SchemaProcessCommand with DataProcessCommand {
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-    processData(sparkSession)
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val dbName = getDB.getDatabaseName(databaseNameOp, sparkSession)
-    val identifier = TableIdentifier(tableName, Option(dbName))
-    val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName, "")
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
-    val carbonEnv = CarbonEnv.getInstance(sparkSession)
-    val catalog = carbonEnv.carbonMetastore
-    val storePath = carbonEnv.storePath
-    val tableIdentifier =
-      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath,
-        dbName.toLowerCase, tableName.toLowerCase)
-    catalog.checkSchemasModifiedTimeAndReloadTables(tableIdentifier.getStorePath)
-    val carbonLocks: scala.collection.mutable.ListBuffer[ICarbonLock] = ListBuffer()
-    try {
-       locksToBeAcquired foreach {
-        lock => carbonLocks += CarbonLockUtil.getLockObject(carbonTableIdentifier, lock)
-      }
-      LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
-
-      CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .dropTable(tableIdentifier.getTablePath, identifier)(sparkSession)
-      LOGGER.audit(s"Deleted table [$tableName] under database [$dbName]")
-    } catch {
-      case ex: Exception =>
-        LOGGER.error(ex, s"Dropping table $dbName.$tableName failed")
-        sys.error(s"Dropping table $dbName.$tableName failed: ${ex.getMessage}")
-    } finally {
-      if (carbonLocks.nonEmpty) {
-        val unlocked = carbonLocks.forall(_.unlock())
-        if (unlocked) {
-          logInfo("Table MetaData Unlocked Successfully")
-        }
-      }
-    }
-    Seq.empty
-  }
-
-  override def processData(sparkSession: SparkSession): Seq[Row] = {
-    // delete the table folder
-    val dbName = getDB.getDatabaseName(databaseNameOp, sparkSession)
-    val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    val tableIdentifier =
-      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath, dbName, tableName)
-    val metadataFilePath =
-      CarbonStorePath.getCarbonTablePath(tableIdentifier).getMetadataDirectoryPath
-    val fileType = FileFactory.getFileType(metadataFilePath)
-    if (FileFactory.isFileExist(metadataFilePath, fileType)) {
-      val file = FileFactory.getCarbonFile(metadataFilePath, fileType)
-      CarbonUtil.deleteFoldersAndFilesSilent(file.getParentFile)
-    }
-    Seq.empty
-  }
-}
-
-private[sql] case class DescribeCommandFormatted(
-    child: SparkPlan,
-    override val output: Seq[Attribute],
-    tblIdentifier: TableIdentifier)
-  extends RunnableCommand with SchemaProcessCommand {
-
-  override def run(sparkSession: SparkSession): Seq[Row] = {
-    processSchema(sparkSession)
-  }
-
-  private def getColumnGroups(dimensions: List[CarbonDimension]): Seq[(String, String, String)] = {
-    var results: Seq[(String, String, String)] =
-      Seq(("", "", ""), ("##Column Group Information", "", ""))
-    val groupedDimensions = dimensions.groupBy(x => x.columnGroupId()).filter {
-      case (groupId, _) => groupId != -1
-    }.toSeq.sortBy(_._1)
-    val groups = groupedDimensions.map(colGroups => {
-      colGroups._2.map(dim => dim.getColName).mkString(", ")
-    })
-    var index = 1
-    groups.foreach { x =>
-      results = results :+ (s"Column Group $index", x, "")
-      index = index + 1
-    }
-    results
-  }
-
-  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
-    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
-        .lookupRelation(tblIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
-    val mapper = new ObjectMapper()
-    val colProps = StringBuilder.newBuilder
-    val dims = relation.metaData.dims.map(x => x.toLowerCase)
-    var results: Seq[(String, String, String)] = child.schema.fields.map { field =>
-      val fieldName = field.name.toLowerCase
-      val comment = if (dims.contains(fieldName)) {
-        val dimension = relation.metaData.carbonTable.getDimensionByName(
-          relation.tableMeta.carbonTableIdentifier.getTableName, fieldName)
-        if (null != dimension.getColumnProperties && !dimension.getColumnProperties.isEmpty) {
-          colProps.append(fieldName).append(".")
-              .append(mapper.writeValueAsString(dimension.getColumnProperties))
-              .append(",")
-        }
-        if (dimension.hasEncoding(Encoding.DICTIONARY) &&
-            !dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          "DICTIONARY, KEY COLUMN" + (dimension.hasEncoding(Encoding.INVERTED_INDEX) match {
-            case false => ",NOINVERTEDINDEX"
-            case _ => ""
-          })
-        } else {
-          "KEY COLUMN" + (dimension.hasEncoding(Encoding.INVERTED_INDEX) match {
-            case false => ",NOINVERTEDINDEX"
-            case _ => ""
-          })
-        }
-      } else {
-        "MEASURE"
-      }
-      (field.name, field.dataType.simpleString, comment)
-    }
-    val colPropStr = if (colProps.toString().trim().length() > 0) {
-      // drops additional comma at end
-      colProps.toString().dropRight(1)
-    } else {
-      colProps.toString()
-    }
-    results ++= Seq(("", "", ""), ("##Detailed Table Information", "", ""))
-    results ++= Seq(("Database Name: ", relation.tableMeta.carbonTableIdentifier
-        .getDatabaseName, "")
-    )
-    results ++= Seq(("Table Name: ", relation.tableMeta.carbonTableIdentifier.getTableName, ""))
-    results ++= Seq(("CARBON Store Path: ", relation.tableMeta.storePath, ""))
-    val carbonTable = relation.tableMeta.carbonTable
-    results ++= Seq(("Table Block Size : ", carbonTable.getBlockSizeInMB + " MB", ""))
-    results ++= Seq(("SORT_SCOPE", carbonTable.getTableInfo.getFactTable
-      .getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants
-      .LOAD_SORT_SCOPE_DEFAULT), CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
-    results ++= Seq(("", "", ""), ("##Detailed Column property", "", ""))
-    if (colPropStr.length() > 0) {
-      results ++= Seq((colPropStr, "", ""))
-    } else {
-      results ++= Seq(("ADAPTIVE", "", ""))
-    }
-    results ++= Seq(("SORT_COLUMNS", relation.metaData.carbonTable.getSortColumns(
-      relation.tableMeta.carbonTableIdentifier.getTableName).asScala
-        .map(column => column).mkString(","), ""))
-    val dimension = carbonTable
-        .getDimensionByTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
-    results ++= getColumnGroups(dimension.asScala.toList)
-    if (carbonTable.getPartitionInfo(carbonTable.getFactTableName) != null) {
-      results ++=
-          Seq(("Partition Columns: ", carbonTable.getPartitionInfo(carbonTable.getFactTableName)
-              .getColumnSchemaList.asScala.map(_.getColumnName).mkString(","), ""))
-    }
-    results.map { case (name, dataType, comment) =>
-      Row(f"$name%-36s", f"$dataType%-80s", f"$comment%-72s")
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
new file mode 100644
index 0000000..28c53a1
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/AlterTableCompactionCommand.scala
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.management
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableModel, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+
+/**
+ * Command for the compaction in alter table command
+ */
+case class AlterTableCompactionCommand(
+    alterTableModel: AlterTableModel)
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+    val tableName = alterTableModel.tableName.toLowerCase
+    val databaseName = alterTableModel.dbName.getOrElse(sparkSession.catalog.currentDatabase)
+    val relation =
+      CarbonEnv.getInstance(sparkSession).carbonMetastore
+        .lookupRelation(Option(databaseName), tableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      sys.error(s"Table $databaseName.$tableName does not exist")
+    }
+    if (null == relation.tableMeta.carbonTable) {
+      LOGGER.error(s"alter table failed. table not found: $databaseName.$tableName")
+      sys.error(s"alter table failed. table not found: $databaseName.$tableName")
+    }
+
+    val carbonLoadModel = new CarbonLoadModel()
+
+    val table = relation.tableMeta.carbonTable
+    carbonLoadModel.setTableName(table.getFactTableName)
+    val dataLoadSchema = new CarbonDataLoadSchema(table)
+    // Need to fill dimension relation
+    carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+    carbonLoadModel.setTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
+    carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
+    carbonLoadModel.setStorePath(relation.tableMeta.carbonTable.getStorePath)
+
+    var storeLocation = CarbonProperties.getInstance
+      .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
+        System.getProperty("java.io.tmpdir")
+      )
+    storeLocation = storeLocation + "/carbonstore/" + System.nanoTime()
+    try {
+      CarbonDataRDDFactory
+        .alterTableForCompaction(sparkSession.sqlContext,
+          alterTableModel,
+          carbonLoadModel,
+          storeLocation
+        )
+    } catch {
+      case e: Exception =>
+        if (null != e.getMessage) {
+          sys.error(s"Compaction failed. Please check logs for more info. ${ e.getMessage }")
+        } else {
+          sys.error("Exception in compaction. Please check logs for more info.")
+        }
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
new file mode 100644
index 0000000..2003bb1
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowLoadsCommand.scala
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.management
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.api.CarbonStore
+
+case class CarbonShowLoadsCommand(
+    databaseNameOp: Option[String],
+    tableName: String,
+    limit: Option[String],
+    override val output: Seq[Attribute])
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
+    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
+      tableMeta.carbonTable
+    CarbonStore.showSegments(
+      GetDB.getDatabaseName(databaseNameOp, sparkSession),
+      tableName,
+      limit,
+      carbonTable.getMetaDataFilepath
+    )
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
new file mode 100644
index 0000000..9406335
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CleanFilesCommand.scala
@@ -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.spark.sql.execution.command.management
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.api.CarbonStore
+
+case class CleanFilesCommand(
+    databaseNameOp: Option[String],
+    tableName: String, forceTableClean: Boolean = false)
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
+    if (forceTableClean) {
+      CarbonStore.cleanFiles(
+        GetDB.getDatabaseName(databaseNameOp, sparkSession),
+        tableName,
+        CarbonEnv.getInstance(sparkSession).storePath,
+        null,
+        forceTableClean)
+    } else {
+      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val relation = catalog
+        .lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation]
+      val carbonTable = relation.tableMeta.carbonTable
+      CarbonStore.cleanFiles(
+        GetDB.getDatabaseName(databaseNameOp, sparkSession),
+        tableName,
+        relation.asInstanceOf[CarbonRelation].tableMeta.storePath,
+        carbonTable,
+        forceTableClean)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
new file mode 100644
index 0000000..1ea4508
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByIdCommand.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.management
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.api.CarbonStore
+
+case class DeleteLoadByIdCommand(
+    loadIds: Seq[String],
+    databaseNameOp: Option[String],
+    tableName: String) extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
+    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
+      tableMeta.carbonTable
+    CarbonStore.deleteLoadById(
+      loadIds,
+      GetDB.getDatabaseName(databaseNameOp, sparkSession),
+      tableName,
+      carbonTable
+    )
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
new file mode 100644
index 0000000..3d06b18
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/DeleteLoadByLoadDateCommand.scala
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.management
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{Checker, DataProcessCommand, RunnableCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.api.CarbonStore
+
+case class DeleteLoadByLoadDateCommand(
+    databaseNameOp: Option[String],
+    tableName: String,
+    dateField: String,
+    loadDate: String)
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
+    val carbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      lookupRelation(databaseNameOp, tableName)(sparkSession).asInstanceOf[CarbonRelation].
+      tableMeta.carbonTable
+    CarbonStore.deleteLoadByDate(
+      loadDate,
+      GetDB.getDatabaseName(databaseNameOp, sparkSession),
+      tableName,
+      carbonTable
+    )
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableByInsertCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableByInsertCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableByInsertCommand.scala
new file mode 100644
index 0000000..3f0e093
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/LoadTableByInsertCommand.scala
@@ -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.spark.sql.execution.command.management
+
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, Dataset, Row, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.command.{DataProcessCommand, RunnableCommand}
+
+import org.apache.carbondata.spark.util.CarbonSparkUtil
+
+case class LoadTableByInsertCommand(
+    relation: CarbonDatasourceHadoopRelation,
+    child: LogicalPlan,
+    overwrite: Boolean)
+  extends RunnableCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processData(sparkSession)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val df = Dataset.ofRows(sparkSession, child)
+    val header = relation.tableSchema.get.fields.map(_.name).mkString(",")
+    val load = LoadTableCommand(
+      Some(relation.carbonRelation.databaseName),
+      relation.carbonRelation.tableName,
+      null,
+      Seq(),
+      scala.collection.immutable.Map("fileheader" -> header),
+      overwrite,
+      null,
+      Some(df)).run(sparkSession)
+    // updating relation metadata. This is in case of auto detect high cardinality
+    relation.carbonRelation.metaData =
+      CarbonSparkUtil.createSparkMeta(relation.carbonRelation.tableMeta.carbonTable)
+    load
+  }
+}


[16/50] [abbrv] carbondata git commit: [CARBONDATA-1504] Fixed refresh of segments in datamap for update and partition

Posted by ja...@apache.org.
[CARBONDATA-1504] Fixed refresh of segments in datamap for update and partition

Currently datamap scans complete segment every time for query execution to get all the carbonindex files. It will be slow when data/number of files are big.
So this PR caches the content of segment and refreshes when any updation or store changes.

This closes #1377


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

Branch: refs/heads/streaming_ingest
Commit: eb771f5ee6511288e3c53281f3fac02d6f69c66a
Parents: c4f312f
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Thu Sep 21 19:03:06 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Sep 28 14:28:44 2017 +0800

----------------------------------------------------------------------
 .../core/datamap/DataMapStoreManager.java       |  89 +-
 .../blockletindex/BlockletDataMap.java          |  14 +-
 .../blockletindex/BlockletDataMapFactory.java   |   9 +-
 .../hadoop/api/CarbonTableInputFormat.java      |  23 +-
 .../testsuite/datamap/DataMapWriterSuite.scala  |   4 +-
 .../iud/UpdateCarbonTableTestCase.scala         | 804 +++++++++----------
 .../apache/spark/sql/hive/CarbonMetastore.scala |   6 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  11 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |   2 +-
 .../spark/sql/hive/CarbonHiveMetaStore.scala    |   2 +-
 10 files changed, 540 insertions(+), 424 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/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 9974920..f19e733 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
@@ -17,6 +17,7 @@
 package org.apache.carbondata.core.datamap;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
@@ -25,6 +26,9 @@ 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.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
+import org.apache.carbondata.core.mutate.UpdateVO;
+import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
 
 /**
  * It maintains all the DataMaps in it.
@@ -38,6 +42,8 @@ public final class DataMapStoreManager {
    */
   private Map<String, List<TableDataMap>> allDataMaps = new ConcurrentHashMap<>();
 
+  private Map<String, TableSegmentRefresher> segmentRefreshMap = new ConcurrentHashMap<>();
+
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(DataMapStoreManager.class.getName());
 
@@ -87,6 +93,8 @@ public final class DataMapStoreManager {
   public TableDataMap createAndRegisterDataMap(AbsoluteTableIdentifier identifier,
       String factoryClassName, String dataMapName) {
     String table = identifier.uniqueName();
+    // Just update the segmentRefreshMap with the table if not added.
+    getTableSegmentRefresher(identifier);
     List<TableDataMap> tableDataMaps = allDataMaps.get(table);
     if (tableDataMaps == null) {
       tableDataMaps = new ArrayList<>();
@@ -124,22 +132,22 @@ public final class DataMapStoreManager {
   }
 
   /**
-   * Clear the datamap/datamaps of a mentioned datamap name and table from memory
-   * @param identifier
-   * @param dataMapName
+   * Clear the datamap/datamaps of a table from memory
+   * @param identifier Table identifier
    */
-  public void clearDataMap(AbsoluteTableIdentifier identifier, String dataMapName) {
+  public void clearDataMap(AbsoluteTableIdentifier identifier) {
     List<TableDataMap> tableDataMaps = allDataMaps.get(identifier.uniqueName());
+    segmentRefreshMap.remove(identifier.uniqueName());
     if (tableDataMaps != null) {
       int i = 0;
       for (TableDataMap tableDataMap: tableDataMaps) {
-        if (tableDataMap != null && dataMapName.equals(tableDataMap.getDataMapName())) {
+        if (tableDataMap != null) {
           tableDataMap.clear();
-          tableDataMaps.remove(i);
           break;
         }
         i++;
       }
+      allDataMaps.remove(identifier.uniqueName());
     }
   }
 
@@ -151,4 +159,73 @@ public final class DataMapStoreManager {
     return instance;
   }
 
+  /**
+   * Get the TableSegmentRefresher for the table. If not existed then add one and return.
+   */
+  public TableSegmentRefresher getTableSegmentRefresher(AbsoluteTableIdentifier identifier) {
+    String uniqueName = identifier.uniqueName();
+    if (segmentRefreshMap.get(uniqueName) == null) {
+      segmentRefreshMap.put(uniqueName, new TableSegmentRefresher(identifier));
+    }
+    return segmentRefreshMap.get(uniqueName);
+  }
+
+  /**
+   * Keep track of the segment refresh time.
+   */
+  public static class TableSegmentRefresher {
+
+    // This map stores the latest segment refresh time.So in case of update/delete we check the
+    // time against this map.
+    private Map<String, Long> segmentRefreshTime = new HashMap<>();
+
+    // This map keeps the manual refresh entries from users. It is mainly used for partition
+    // altering.
+    private Map<String, Boolean> manualSegmentRefresh = new HashMap<>();
+
+    public TableSegmentRefresher(AbsoluteTableIdentifier identifier) {
+      SegmentUpdateStatusManager statusManager = new SegmentUpdateStatusManager(identifier);
+      SegmentUpdateDetails[] updateStatusDetails = statusManager.getUpdateStatusDetails();
+      for (SegmentUpdateDetails updateDetails : updateStatusDetails) {
+        UpdateVO updateVO = statusManager.getInvalidTimestampRange(updateDetails.getSegmentName());
+        segmentRefreshTime.put(updateVO.getSegmentId(), updateVO.getCreatedOrUpdatedTimeStamp());
+      }
+    }
+
+    public boolean isRefreshNeeded(String segmentId, SegmentUpdateStatusManager statusManager) {
+      UpdateVO updateVO = statusManager.getInvalidTimestampRange(segmentId);
+      if (segmentRefreshTime.get(segmentId) == null) {
+        segmentRefreshTime.put(segmentId, updateVO.getCreatedOrUpdatedTimeStamp());
+        return true;
+      }
+      if (manualSegmentRefresh.get(segmentId) != null && manualSegmentRefresh.get(segmentId)) {
+        manualSegmentRefresh.put(segmentId, false);
+        return true;
+      }
+      Long updateTimestamp = updateVO.getLatestUpdateTimestamp();
+      boolean isRefresh =
+          updateTimestamp != null && (updateTimestamp > segmentRefreshTime.get(segmentId));
+      if (isRefresh) {
+        segmentRefreshTime.remove(segmentId);
+      }
+      return isRefresh;
+    }
+
+    public void refreshSegments(List<String> segmentIds) {
+      for (String segmentId: segmentIds) {
+        manualSegmentRefresh.put(segmentId, true);
+      }
+    }
+
+    public boolean isRefreshNeeded(String segmentId) {
+      if (manualSegmentRefresh.get(segmentId) != null && manualSegmentRefresh.get(segmentId)) {
+        manualSegmentRefresh.put(segmentId, false);
+        return true;
+      } else {
+        return false;
+      }
+    }
+  }
+
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/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 bd7a1d3..57211fd 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
@@ -429,9 +429,11 @@ public class BlockletDataMap implements DataMap, Cacheable {
 
   @Override
   public void clear() {
-    unsafeMemoryDMStore.freeMemory();
-    unsafeMemoryDMStore = null;
-    segmentProperties = null;
+    if (unsafeMemoryDMStore != null) {
+      unsafeMemoryDMStore.freeMemory();
+      unsafeMemoryDMStore = null;
+      segmentProperties = null;
+    }
   }
 
   @Override
@@ -446,7 +448,11 @@ public class BlockletDataMap implements DataMap, Cacheable {
 
   @Override
   public long getMemorySize() {
-    return unsafeMemoryDMStore.getMemoryUsed();
+    if (unsafeMemoryDMStore != null) {
+      return unsafeMemoryDMStore.getMemoryUsed();
+    } else {
+      return 0;
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/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 d6ddfa6..d734d81 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
@@ -77,6 +77,7 @@ public class BlockletDataMapFactory implements DataMapFactory {
         tableBlockIndexUniqueIdentifiers.add(
             new TableBlockIndexUniqueIdentifier(identifier, segmentId, listFiles[i].getName()));
       }
+      segmentMap.put(segmentId, tableBlockIndexUniqueIdentifiers);
     }
 
     return cache.getAll(tableBlockIndexUniqueIdentifiers);
@@ -120,15 +121,17 @@ public class BlockletDataMapFactory implements DataMapFactory {
     if (blockIndexes != null) {
       for (TableBlockIndexUniqueIdentifier blockIndex : blockIndexes) {
         DataMap dataMap = cache.getIfPresent(blockIndex);
-        dataMap.clear();
-        cache.invalidate(blockIndex);
+        if (dataMap != null) {
+          cache.invalidate(blockIndex);
+          dataMap.clear();
+        }
       }
     }
   }
 
   @Override
   public void clear() {
-    for (String segmentId: segmentMap.keySet()) {
+    for (String segmentId: segmentMap.keySet().toArray(new String[segmentMap.size()])) {
       clear(segmentId);
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/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 9076233..55d22d1 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
@@ -272,16 +272,15 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     List<UpdateVO> invalidTimestampsList = new ArrayList<>();
     List<String> validSegments = Arrays.asList(getSegmentsToAccess(job));
     // get all valid segments and set them into the configuration
+    SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
     if (validSegments.size() == 0) {
       SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
       SegmentStatusManager.ValidAndInvalidSegmentsInfo segments =
           segmentStatusManager.getValidAndInvalidSegments();
-      SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(identifier);
       validSegments = segments.getValidSegments();
       if (validSegments.size() == 0) {
         return new ArrayList<>(0);
       }
-
       // remove entry in the segment index if there are invalid segments
       invalidSegments.addAll(segments.getInvalidSegments());
       for (String invalidSegmentId : invalidSegments) {
@@ -292,6 +291,26 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
       }
     }
 
+    // Clean the updated segments from memory if the update happens on segments
+    List<String> toBeCleanedSegments = new ArrayList<>();
+    for (SegmentUpdateDetails segmentUpdateDetail : updateStatusManager
+        .getUpdateStatusDetails()) {
+      boolean refreshNeeded =
+          DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
+              .isRefreshNeeded(segmentUpdateDetail.getSegmentName(), updateStatusManager);
+      if (refreshNeeded) {
+        toBeCleanedSegments.add(segmentUpdateDetail.getSegmentName());
+      }
+    }
+    // Clean segments if refresh is needed
+    for (String segment : validSegments) {
+      if (DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
+          .isRefreshNeeded(segment)) {
+        toBeCleanedSegments.add(segment);
+      }
+    }
+    blockletMap.clear(toBeCleanedSegments);
+
     // process and resolve the expression
     Expression filter = getFilterPredicates(job.getConfiguration());
     CarbonTable carbonTable = getOrCreateCarbonTable(job.getConfiguration());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/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 e8ec6ad..0b5141e 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
@@ -41,9 +41,9 @@ class C2DataMapFactory() extends DataMapFactory {
 
   override def fireEvent(event: ChangeEvent[_]): Unit = ???
 
-  override def clear(segmentId: String): Unit = ???
+  override def clear(segmentId: String): Unit = {}
 
-  override def clear(): Unit = ???
+  override def clear(): Unit = {}
 
   override def getDataMap(distributable: DataMapDistributable): DataMap = ???
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index 4814183..db289d9 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -45,408 +45,408 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
   }
 
 
-//  test("test update operation with 0 rows updation.") {
-//    sql("""drop table if exists iud.zerorows""").show
-//    sql("""create table iud.zerorows (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.zerorows""")
-//    sql("""update zerorows d  set (d.c2) = (d.c2 + 1) where d.c1 = 'a'""").show()
-//    sql("""update zerorows d  set (d.c2) = (d.c2 + 1) where d.c1 = 'xxx'""").show()
-//    checkAnswer(
-//      sql("""select c1,c2,c3,c5 from iud.zerorows"""),
-//      Seq(Row("a",2,"aa","aaa"),Row("b",2,"bb","bbb"),Row("c",3,"cc","ccc"),Row("d",4,"dd","ddd"),Row("e",5,"ee","eee"))
-//    )
-//    sql("""drop table iud.zerorows""").show
-//
-//
-//  }
-//
-//
-//  test("update carbon table[select from source table with where and exist]") {
-//    sql("""drop table if exists iud.dest11""").show
-//    sql("""create table iud.dest11 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest11""")
-//    sql("""update iud.dest11 d set (d.c3, d.c5 ) = (select s.c33,s.c55 from iud.source2 s where d.c1 = s.c11) where 1 = 1""").show()
-//    checkAnswer(
-//      sql("""select c3,c5 from iud.dest11"""),
-//      Seq(Row("cc","ccc"), Row("dd","ddd"),Row("ee","eee"), Row("MGM","Disco"),Row("RGK","Music"))
-//    )
-//    sql("""drop table iud.dest11""").show
-//  }
-//
-//  test("update carbon table[using destination table columns with where and exist]") {
-//    sql("""drop table if exists iud.dest22""")
-//    sql("""create table iud.dest22 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest22""")
-//    checkAnswer(
-//      sql("""select c2 from iud.dest22 where c1='a'"""),
-//      Seq(Row(1))
-//    )
-//    sql("""update dest22 d  set (d.c2) = (d.c2 + 1) where d.c1 = 'a'""").show()
-//    checkAnswer(
-//      sql("""select c2 from iud.dest22 where c1='a'"""),
-//      Seq(Row(2))
-//    )
-//    sql("""drop table if exists iud.dest22""")
-//  }
-//
-//  test("update carbon table without alias in set columns") {
-//    sql("""drop table if exists iud.dest33""")
-//    sql("""create table iud.dest33 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest33""")
-//    sql("""update iud.dest33 d set (c3,c5 ) = (select s.c33 ,s.c55  from iud.source2 s where d.c1 = s.c11) where d.c1 = 'a'""").show()
-//    checkAnswer(
-//      sql("""select c3,c5 from iud.dest33 where c1='a'"""),
-//      Seq(Row("MGM","Disco"))
-//    )
-//    sql("""drop table if exists iud.dest33""")
-//  }
-//
-//  test("update carbon table without alias in set columns with mulitple loads") {
-//    sql("""drop table if exists iud.dest33""")
-//    sql("""create table iud.dest33 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest33""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest33""")
-//    sql("""update iud.dest33 d set (c3,c5 ) = (select s.c33 ,s.c55  from iud.source2 s where d.c1 = s.c11) where d.c1 = 'a'""").show()
-//    checkAnswer(
-//      sql("""select c3,c5 from iud.dest33 where c1='a'"""),
-//      Seq(Row("MGM","Disco"),Row("MGM","Disco"))
-//    )
-//    sql("""drop table if exists iud.dest33""")
-//  }
-//
-//  test("update carbon table with optimized parallelism for segment") {
-//    sql("""drop table if exists iud.dest_opt_segment_parallelism""")
-//    sql(
-//      """create table iud.dest_opt_segment_parallelism (c1 string,c2 int,c3 string,c5 string)
-//        | STORED BY 'org.apache.carbondata.format'""".stripMargin)
-//    sql(
-//      s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv'
-//         | INTO table iud.dest_opt_segment_parallelism""".stripMargin)
-//    sql(
-//      s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv'
-//         | INTO table iud.dest_opt_segment_parallelism""".stripMargin)
-//    CarbonProperties.getInstance().addProperty(
-//      CarbonCommonConstants.CARBON_UPDATE_SEGMENT_PARALLELISM, "3")
-//    sql(
-//      """update iud.dest_opt_segment_parallelism d
-//        | set (c3,c5 ) = (select s.c33 ,s.c55 from iud.source2 s where d.c1 = s.c11)
-//        | where d.c1 = 'a'""".stripMargin).show()
-//    checkAnswer(
-//      sql("""select c3,c5 from iud.dest_opt_segment_parallelism where c1='a'"""),
-//      Seq(Row("MGM","Disco"),Row("MGM","Disco"))
-//    )
-//    sql("""drop table if exists iud.dest_opt_segment_parallelism""")
-//  }
-//
-//  test("update carbon table without alias in set three columns") {
-//    sql("""drop table if exists iud.dest44""")
-//    sql("""create table iud.dest44 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest44""")
-//    sql("""update iud.dest44 d set (c1,c3,c5 ) = (select s.c11, s.c33 ,s.c55  from iud.source2 s where d.c1 = s.c11) where d.c1 = 'a'""").show()
-//    checkAnswer(
-//      sql("""select c1,c3,c5 from iud.dest44 where c1='a'"""),
-//      Seq(Row("a","MGM","Disco"))
-//    )
-//    sql("""drop table if exists iud.dest44""")
-//  }
-//
-//  test("update carbon table[single column select from source with where and exist]") {
-//    sql("""drop table if exists iud.dest55""")
-//    sql("""create table iud.dest55 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest55""")
-//    sql("""update iud.dest55 d set (c3)  = (select s.c33 from iud.source2 s where d.c1 = s.c11) where 1 = 1""").show()
-//    checkAnswer(
-//      sql("""select c1,c3 from iud.dest55 """),
-//      Seq(Row("a","MGM"),Row("b","RGK"),Row("c","cc"),Row("d","dd"),Row("e","ee"))
-//    )
-//    sql("""drop table if exists iud.dest55""")
-//  }
-//
-//  test("update carbon table[single column SELECT from source with where and exist]") {
-//    sql("""drop table if exists iud.dest55""")
-//    sql("""create table iud.dest55 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest55""")
-//    sql("""update iud.dest55 d set (c3)  = (SELECT s.c33 from iud.source2 s where d.c1 = s.c11) where 1 = 1""").show()
-//    checkAnswer(
-//      sql("""select c1,c3 from iud.dest55 """),
-//      Seq(Row("a","MGM"),Row("b","RGK"),Row("c","cc"),Row("d","dd"),Row("e","ee"))
-//    )
-//    sql("""drop table if exists iud.dest55""")
-//  }
-//
-//  test("update carbon table[using destination table columns without where clause]") {
-//    sql("""drop table if exists iud.dest66""")
-//    sql("""create table iud.dest66 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest66""")
-//    sql("""update iud.dest66 d set (c2, c5 ) = (c2 + 1, concat(c5 , "z"))""").show()
-//    checkAnswer(
-//      sql("""select c2,c5 from iud.dest66 """),
-//      Seq(Row(2,"aaaz"),Row(3,"bbbz"),Row(4,"cccz"),Row(5,"dddz"),Row(6,"eeez"))
-//    )
-//    sql("""drop table if exists iud.dest66""")
-//  }
-//
-//  test("update carbon table[using destination table columns with where clause]") {
-//    sql("""drop table if exists iud.dest77""")
-//    sql("""create table iud.dest77 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest77""")
-//    sql("""update iud.dest77 d set (c2, c5 ) = (c2 + 1, concat(c5 , "z")) where d.c3 = 'dd'""").show()
-//    checkAnswer(
-//      sql("""select c2,c5 from iud.dest77 where c3 = 'dd'"""),
-//      Seq(Row(5,"dddz"))
-//    )
-//    sql("""drop table if exists iud.dest77""")
-//  }
-//
-//  test("update carbon table[using destination table( no alias) columns without where clause]") {
-//    sql("""drop table if exists iud.dest88""")
-//    sql("""create table iud.dest88 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest88""")
-//    sql("""update iud.dest88  set (c2, c5 ) = (c2 + 1, concat(c5 , "y" ))""").show()
-//    checkAnswer(
-//      sql("""select c2,c5 from iud.dest88 """),
-//      Seq(Row(2,"aaay"),Row(3,"bbby"),Row(4,"cccy"),Row(5,"dddy"),Row(6,"eeey"))
-//    )
-//    sql("""drop table if exists iud.dest88""")
-//  }
-//
-//  test("update carbon table[using destination table columns with hard coded value ]") {
-//    sql("""drop table if exists iud.dest99""")
-//    sql("""create table iud.dest99 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest99""")
-//    sql("""update iud.dest99 d set (c2, c5 ) = (c2 + 1, "xyx")""").show()
-//    checkAnswer(
-//      sql("""select c2,c5 from iud.dest99 """),
-//      Seq(Row(2,"xyx"),Row(3,"xyx"),Row(4,"xyx"),Row(5,"xyx"),Row(6,"xyx"))
-//    )
-//    sql("""drop table if exists iud.dest99""")
-//  }
-//
-//  test("update carbon tableusing destination table columns with hard coded value and where condition]") {
-//    sql("""drop table if exists iud.dest110""")
-//    sql("""create table iud.dest110 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest110""")
-//    sql("""update iud.dest110 d set (c2, c5 ) = (c2 + 1, "xyx") where d.c1 = 'e'""").show()
-//    checkAnswer(
-//      sql("""select c2,c5 from iud.dest110 where c1 = 'e' """),
-//      Seq(Row(6,"xyx"))
-//    )
-//    sql("""drop table iud.dest110""")
-//  }
-//
-//  test("update carbon table[using source  table columns with where and exist and no destination table condition]") {
-//    sql("""drop table if exists iud.dest120""")
-//    sql("""create table iud.dest120 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest120""")
-//    sql("""update iud.dest120 d  set (c3, c5 ) = (select s.c33 ,s.c55  from iud.source2 s where d.c1 = s.c11)""").show()
-//    checkAnswer(
-//      sql("""select c3,c5 from iud.dest120 """),
-//      Seq(Row("MGM","Disco"),Row("RGK","Music"),Row("cc","ccc"),Row("dd","ddd"),Row("ee","eee"))
-//    )
-//    sql("""drop table iud.dest120""")
-//  }
-//
-//  test("update carbon table[using destination table where and exist]") {
-//    sql("""drop table if exists iud.dest130""")
-//    sql("""create table iud.dest130 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest130""")
-//    sql("""update iud.dest130 dd  set (c2, c5 ) = (c2 + 1, "xyx")  where dd.c1 = 'a'""").show()
-//    checkAnswer(
-//      sql("""select c2,c5 from iud.dest130 where c1 = 'a' """),
-//      Seq(Row(2,"xyx"))
-//    )
-//    sql("""drop table iud.dest130""")
-//  }
-//
-//  test("update carbon table[using destination table (concat) where and exist]") {
-//    sql("""drop table if exists iud.dest140""")
-//    sql("""create table iud.dest140 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest140""")
-//    sql("""update iud.dest140 d set (c2, c5 ) = (c2 + 1, concat(c5 , "z"))  where d.c1 = 'a'""").show()
-//    checkAnswer(
-//      sql("""select c2,c5 from iud.dest140 where c1 = 'a'"""),
-//      Seq(Row(2,"aaaz"))
-//    )
-//    sql("""drop table iud.dest140""")
-//  }
-//
-//  test("update carbon table[using destination table (concat) with  where") {
-//    sql("""drop table if exists iud.dest150""")
-//    sql("""create table iud.dest150 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest150""")
-//    sql("""update iud.dest150 d set (c5) = (concat(c5 , "z"))  where d.c1 = 'b'""").show()
-//    checkAnswer(
-//      sql("""select c5 from iud.dest150 where c1 = 'b' """),
-//      Seq(Row("bbbz"))
-//    )
-//    sql("""drop table iud.dest150""")
-//  }
-//
-//  test("update table with data for datatype mismatch with column ") {
-//    sql("""update iud.update_01 set (imei) = ('skt') where level = 'aaa'""")
-//    checkAnswer(
-//      sql("""select * from iud.update_01 where imei = 'skt'"""),
-//      Seq()
-//    )
-//  }
-//
-//  test("update carbon table-error[more columns in source table not allowed") {
-//    val exception = intercept[Exception] {
-//      sql("""update iud.dest d set (c2, c5 ) = (c2 + 1, concat(c5 , "z"), "abc")""").show()
-//    }
-//    assertResult("Number of source and destination columns are not matching")(exception.getMessage)
-//  }
-//
-//  test("update carbon table-error[no set columns") {
-//    intercept[Exception] {
-//      sql("""update iud.dest d set () = ()""").show()
-//    }
-//  }
-//
-//  test("update carbon table-error[no set columns with updated column") {
-//    intercept[Exception] {
-//      sql("""update iud.dest d set  = (c1+1)""").show()
-//    }
-//  }
-//  test("update carbon table-error[one set column with two updated column") {
-//    intercept[Exception] {
-//      sql("""update iud.dest  set c2 = (c2 + 1, concat(c5 , "z") )""").show()
-//    }
-//  }
-//
-//  test("""update carbon [special characters  in value- test parsing logic ]""") {
-//    sql("""drop table if exists iud.dest160""")
-//    sql("""create table iud.dest160 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest160""")
-//    sql("""update iud.dest160 set(c1) = ("ab\')$*)(&^)")""").show()
-//    sql("""update iud.dest160 set(c1) =  ('abd$asjdh$adasj$l;sdf$*)$*)(&^')""").show()
-//    sql("""update iud.dest160 set(c1) =("\\")""").show()
-//    sql("""update iud.dest160 set(c1) = ("ab\')$*)(&^)")""").show()
-//    sql("""update iud.dest160 d set (c3,c5)=(select s.c33,'a\\a' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
-//    sql("""update iud.dest160 d set (c3,c5)=(select s.c33,'\\' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
-//    sql("""update iud.dest160 d set (c3,c5)=(select s.c33,'\\a' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
-//    sql("""update iud.dest160 d set (c3,c5)      =     (select s.c33,'a\\a\\' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
-//    sql("""update iud.dest160 d set (c3,c5) =(select s.c33,'a\'a\\' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
-//    sql("""update iud.dest160 d set (c3,c5)=(select s.c33,'\\a\'a\"' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
-//    sql("""drop table iud.dest160""")
-//  }
-//
-//  test("""update carbon [sub query, between and existing in outer condition.(Customer query ) ]""") {
-//    sql("""drop table if exists iud.dest170""")
-//    sql("""create table iud.dest170 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest170""")
-//    sql("""update iud.dest170 d set (c3)=(select s.c33 from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
-//    checkAnswer(
-//      sql("""select c3 from  iud.dest170 as d where d.c2 between 1 and 3"""),
-//      Seq(Row("MGM"), Row("RGK"), Row("cc"))
-//    )
-//    sql("""drop table iud.dest170""")
-//  }
-//
-//  test("""update carbon [self join select query ]""") {
-//    sql("""drop table if exists iud.dest171""")
-//    sql("""create table iud.dest171 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest171""")
-//    sql("""update iud.dest171 d set (c3)=(select concat(s.c3 , "z") from iud.dest171 s where d.c2 = s.c2)""").show
-//    sql("""drop table if exists iud.dest172""")
-//    sql("""create table iud.dest172 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest172""")
-//    sql("""update iud.dest172 d set (c3)=( concat(c3 , "z"))""").show
-//    checkAnswer(
-//      sql("""select c3 from  iud.dest171"""),
-//      sql("""select c3 from  iud.dest172""")
-//    )
-//    sql("""drop table iud.dest171""")
-//    sql("""drop table iud.dest172""")
-//  }
-//
-//  test("update carbon table-error[closing bracket missed") {
-//    intercept[Exception] {
-//      sql("""update iud.dest d set (c2) = (194""").show()
-//    }
-//  }
-//
-//  test("update carbon table-error[starting bracket missed") {
-//    intercept[Exception] {
-//      sql("""update iud.dest d set (c2) = 194)""").show()
-//    }
-//  }
-//
-//  test("update carbon table-error[missing starting and closing bracket") {
-//    intercept[Exception] {
-//      sql("""update iud.dest d set (c2) = 194""").show()
-//    }
-//  }
-//
-//  test("test create table with column name as tupleID"){
-//    intercept[Exception] {
-//      sql("CREATE table carbontable (empno int, tupleID String, " +
-//          "designation String, doj Timestamp, workgroupcategory int, " +
-//          "workgroupcategoryname String, deptno int, deptname String, projectcode int, " +
-//          "projectjoindate Timestamp, projectenddate Timestamp, attendance int, " +
-//          "utilization int,salary int) STORED BY 'org.apache.carbondata.format' " +
-//          "TBLPROPERTIES('DICTIONARY_INCLUDE'='empno,workgroupcategory,deptno,projectcode'," +
-//          "'DICTIONARY_EXCLUDE'='empname')")
-//    }
-//  }
-//
-//  test("test show segment after updating data : JIRA-1411,JIRA-1414") {
-//    sql("""drop table if exists iud.show_segment""").show
-//    sql("""create table iud.show_segment (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
-//    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.show_segment""")
-//    val before_update = sql("""show segments for table iud.show_segment""").toDF()
-//    sql("""update iud.show_segment d set (d.c3, d.c5 ) = (select s.c33,s.c55 from iud.source2 s where d.c1 = s.c11) where 1 = 1""").show()
-//    val after_update = sql("""show segments for table iud.show_segment""").toDF()
-//    checkAnswer(
-//      before_update,
-//      after_update
-//    )
-//    sql("""drop table if exists iud.show_segment""").show
-//  }
-//
-//  test("Failure of update operation due to bad record with proper error message") {
-//    try {
-//      CarbonProperties.getInstance()
-//        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FAIL")
-//      val errorMessage = intercept[Exception] {
-//        sql("drop table if exists update_with_bad_record")
-//        sql("create table update_with_bad_record(item int, name String) stored by 'carbondata'")
-//        sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/IUD/bad_record.csv' into table " +
-//            s"update_with_bad_record")
-//        sql("update update_with_bad_record set (item)=(3.45)").show()
-//        sql("drop table if exists update_with_bad_record")
-//      }
-//      assert(errorMessage.getMessage.contains("Data load failed due to bad record"))
-//    } finally {
-//      CarbonProperties.getInstance()
-//        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FORCE")
-//    }
-//  }
-//
-//  test("More records after update operation ") {
-//    sql("DROP TABLE IF EXISTS default.carbon1")
-//    import sqlContext.implicits._
-//    val df = sqlContext.sparkContext.parallelize(1 to 36000)
-//      .map(x => (x+"a", "b", x))
-//      .toDF("c1", "c2", "c3")
-//    df.write
-//      .format("carbondata")
-//      .option("tableName", "carbon1")
-//      .option("tempCSV", "true")
-//      .option("compress", "true")
-//      .mode(SaveMode.Overwrite)
-//      .save()
-//
-//    checkAnswer(sql("select count(*) from default.carbon1"), Seq(Row(36000)))
-//
-//    sql("update default.carbon1 set (c1)=('test123') where c1='9999a'").show()
-//
-//    checkAnswer(sql("select count(*) from default.carbon1"), Seq(Row(36000)))
-//
-//    checkAnswer(sql("select * from default.carbon1 where c1 = 'test123'"), Row("test123","b",9999))
-//
-//    sql("DROP TABLE IF EXISTS default.carbon1")
-//  }
+  test("test update operation with 0 rows updation.") {
+    sql("""drop table if exists iud.zerorows""").show
+    sql("""create table iud.zerorows (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.zerorows""")
+    sql("""update zerorows d  set (d.c2) = (d.c2 + 1) where d.c1 = 'a'""").show()
+    sql("""update zerorows d  set (d.c2) = (d.c2 + 1) where d.c1 = 'xxx'""").show()
+    checkAnswer(
+      sql("""select c1,c2,c3,c5 from iud.zerorows"""),
+      Seq(Row("a",2,"aa","aaa"),Row("b",2,"bb","bbb"),Row("c",3,"cc","ccc"),Row("d",4,"dd","ddd"),Row("e",5,"ee","eee"))
+    )
+    sql("""drop table iud.zerorows""").show
+
+
+  }
+
+
+  test("update carbon table[select from source table with where and exist]") {
+    sql("""drop table if exists iud.dest11""").show
+    sql("""create table iud.dest11 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest11""")
+    sql("""update iud.dest11 d set (d.c3, d.c5 ) = (select s.c33,s.c55 from iud.source2 s where d.c1 = s.c11) where 1 = 1""").show()
+    checkAnswer(
+      sql("""select c3,c5 from iud.dest11"""),
+      Seq(Row("cc","ccc"), Row("dd","ddd"),Row("ee","eee"), Row("MGM","Disco"),Row("RGK","Music"))
+    )
+    sql("""drop table iud.dest11""").show
+  }
+
+  test("update carbon table[using destination table columns with where and exist]") {
+    sql("""drop table if exists iud.dest22""")
+    sql("""create table iud.dest22 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest22""")
+    checkAnswer(
+      sql("""select c2 from iud.dest22 where c1='a'"""),
+      Seq(Row(1))
+    )
+    sql("""update dest22 d  set (d.c2) = (d.c2 + 1) where d.c1 = 'a'""").show()
+    checkAnswer(
+      sql("""select c2 from iud.dest22 where c1='a'"""),
+      Seq(Row(2))
+    )
+    sql("""drop table if exists iud.dest22""")
+  }
+
+  test("update carbon table without alias in set columns") {
+    sql("""drop table if exists iud.dest33""")
+    sql("""create table iud.dest33 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest33""")
+    sql("""update iud.dest33 d set (c3,c5 ) = (select s.c33 ,s.c55  from iud.source2 s where d.c1 = s.c11) where d.c1 = 'a'""").show()
+    checkAnswer(
+      sql("""select c3,c5 from iud.dest33 where c1='a'"""),
+      Seq(Row("MGM","Disco"))
+    )
+    sql("""drop table if exists iud.dest33""")
+  }
+
+  test("update carbon table without alias in set columns with mulitple loads") {
+    sql("""drop table if exists iud.dest33""")
+    sql("""create table iud.dest33 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest33""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest33""")
+    sql("""update iud.dest33 d set (c3,c5 ) = (select s.c33 ,s.c55  from iud.source2 s where d.c1 = s.c11) where d.c1 = 'a'""").show()
+    checkAnswer(
+      sql("""select c3,c5 from iud.dest33 where c1='a'"""),
+      Seq(Row("MGM","Disco"),Row("MGM","Disco"))
+    )
+    sql("""drop table if exists iud.dest33""")
+  }
+
+  test("update carbon table with optimized parallelism for segment") {
+    sql("""drop table if exists iud.dest_opt_segment_parallelism""")
+    sql(
+      """create table iud.dest_opt_segment_parallelism (c1 string,c2 int,c3 string,c5 string)
+        | STORED BY 'org.apache.carbondata.format'""".stripMargin)
+    sql(
+      s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv'
+         | INTO table iud.dest_opt_segment_parallelism""".stripMargin)
+    sql(
+      s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv'
+         | INTO table iud.dest_opt_segment_parallelism""".stripMargin)
+    CarbonProperties.getInstance().addProperty(
+      CarbonCommonConstants.CARBON_UPDATE_SEGMENT_PARALLELISM, "3")
+    sql(
+      """update iud.dest_opt_segment_parallelism d
+        | set (c3,c5 ) = (select s.c33 ,s.c55 from iud.source2 s where d.c1 = s.c11)
+        | where d.c1 = 'a'""".stripMargin).show()
+    checkAnswer(
+      sql("""select c3,c5 from iud.dest_opt_segment_parallelism where c1='a'"""),
+      Seq(Row("MGM","Disco"),Row("MGM","Disco"))
+    )
+    sql("""drop table if exists iud.dest_opt_segment_parallelism""")
+  }
+
+  test("update carbon table without alias in set three columns") {
+    sql("""drop table if exists iud.dest44""")
+    sql("""create table iud.dest44 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest44""")
+    sql("""update iud.dest44 d set (c1,c3,c5 ) = (select s.c11, s.c33 ,s.c55  from iud.source2 s where d.c1 = s.c11) where d.c1 = 'a'""").show()
+    checkAnswer(
+      sql("""select c1,c3,c5 from iud.dest44 where c1='a'"""),
+      Seq(Row("a","MGM","Disco"))
+    )
+    sql("""drop table if exists iud.dest44""")
+  }
+
+  test("update carbon table[single column select from source with where and exist]") {
+    sql("""drop table if exists iud.dest55""")
+    sql("""create table iud.dest55 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest55""")
+    sql("""update iud.dest55 d set (c3)  = (select s.c33 from iud.source2 s where d.c1 = s.c11) where 1 = 1""").show()
+    checkAnswer(
+      sql("""select c1,c3 from iud.dest55 """),
+      Seq(Row("a","MGM"),Row("b","RGK"),Row("c","cc"),Row("d","dd"),Row("e","ee"))
+    )
+    sql("""drop table if exists iud.dest55""")
+  }
+
+  test("update carbon table[single column SELECT from source with where and exist]") {
+    sql("""drop table if exists iud.dest55""")
+    sql("""create table iud.dest55 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest55""")
+    sql("""update iud.dest55 d set (c3)  = (SELECT s.c33 from iud.source2 s where d.c1 = s.c11) where 1 = 1""").show()
+    checkAnswer(
+      sql("""select c1,c3 from iud.dest55 """),
+      Seq(Row("a","MGM"),Row("b","RGK"),Row("c","cc"),Row("d","dd"),Row("e","ee"))
+    )
+    sql("""drop table if exists iud.dest55""")
+  }
+
+  test("update carbon table[using destination table columns without where clause]") {
+    sql("""drop table if exists iud.dest66""")
+    sql("""create table iud.dest66 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest66""")
+    sql("""update iud.dest66 d set (c2, c5 ) = (c2 + 1, concat(c5 , "z"))""").show()
+    checkAnswer(
+      sql("""select c2,c5 from iud.dest66 """),
+      Seq(Row(2,"aaaz"),Row(3,"bbbz"),Row(4,"cccz"),Row(5,"dddz"),Row(6,"eeez"))
+    )
+    sql("""drop table if exists iud.dest66""")
+  }
+
+  test("update carbon table[using destination table columns with where clause]") {
+    sql("""drop table if exists iud.dest77""")
+    sql("""create table iud.dest77 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest77""")
+    sql("""update iud.dest77 d set (c2, c5 ) = (c2 + 1, concat(c5 , "z")) where d.c3 = 'dd'""").show()
+    checkAnswer(
+      sql("""select c2,c5 from iud.dest77 where c3 = 'dd'"""),
+      Seq(Row(5,"dddz"))
+    )
+    sql("""drop table if exists iud.dest77""")
+  }
+
+  test("update carbon table[using destination table( no alias) columns without where clause]") {
+    sql("""drop table if exists iud.dest88""")
+    sql("""create table iud.dest88 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest88""")
+    sql("""update iud.dest88  set (c2, c5 ) = (c2 + 1, concat(c5 , "y" ))""").show()
+    checkAnswer(
+      sql("""select c2,c5 from iud.dest88 """),
+      Seq(Row(2,"aaay"),Row(3,"bbby"),Row(4,"cccy"),Row(5,"dddy"),Row(6,"eeey"))
+    )
+    sql("""drop table if exists iud.dest88""")
+  }
+
+  test("update carbon table[using destination table columns with hard coded value ]") {
+    sql("""drop table if exists iud.dest99""")
+    sql("""create table iud.dest99 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest99""")
+    sql("""update iud.dest99 d set (c2, c5 ) = (c2 + 1, "xyx")""").show()
+    checkAnswer(
+      sql("""select c2,c5 from iud.dest99 """),
+      Seq(Row(2,"xyx"),Row(3,"xyx"),Row(4,"xyx"),Row(5,"xyx"),Row(6,"xyx"))
+    )
+    sql("""drop table if exists iud.dest99""")
+  }
+
+  test("update carbon tableusing destination table columns with hard coded value and where condition]") {
+    sql("""drop table if exists iud.dest110""")
+    sql("""create table iud.dest110 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest110""")
+    sql("""update iud.dest110 d set (c2, c5 ) = (c2 + 1, "xyx") where d.c1 = 'e'""").show()
+    checkAnswer(
+      sql("""select c2,c5 from iud.dest110 where c1 = 'e' """),
+      Seq(Row(6,"xyx"))
+    )
+    sql("""drop table iud.dest110""")
+  }
+
+  test("update carbon table[using source  table columns with where and exist and no destination table condition]") {
+    sql("""drop table if exists iud.dest120""")
+    sql("""create table iud.dest120 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest120""")
+    sql("""update iud.dest120 d  set (c3, c5 ) = (select s.c33 ,s.c55  from iud.source2 s where d.c1 = s.c11)""").show()
+    checkAnswer(
+      sql("""select c3,c5 from iud.dest120 """),
+      Seq(Row("MGM","Disco"),Row("RGK","Music"),Row("cc","ccc"),Row("dd","ddd"),Row("ee","eee"))
+    )
+    sql("""drop table iud.dest120""")
+  }
+
+  test("update carbon table[using destination table where and exist]") {
+    sql("""drop table if exists iud.dest130""")
+    sql("""create table iud.dest130 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest130""")
+    sql("""update iud.dest130 dd  set (c2, c5 ) = (c2 + 1, "xyx")  where dd.c1 = 'a'""").show()
+    checkAnswer(
+      sql("""select c2,c5 from iud.dest130 where c1 = 'a' """),
+      Seq(Row(2,"xyx"))
+    )
+    sql("""drop table iud.dest130""")
+  }
+
+  test("update carbon table[using destination table (concat) where and exist]") {
+    sql("""drop table if exists iud.dest140""")
+    sql("""create table iud.dest140 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest140""")
+    sql("""update iud.dest140 d set (c2, c5 ) = (c2 + 1, concat(c5 , "z"))  where d.c1 = 'a'""").show()
+    checkAnswer(
+      sql("""select c2,c5 from iud.dest140 where c1 = 'a'"""),
+      Seq(Row(2,"aaaz"))
+    )
+    sql("""drop table iud.dest140""")
+  }
+
+  test("update carbon table[using destination table (concat) with  where") {
+    sql("""drop table if exists iud.dest150""")
+    sql("""create table iud.dest150 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest150""")
+    sql("""update iud.dest150 d set (c5) = (concat(c5 , "z"))  where d.c1 = 'b'""").show()
+    checkAnswer(
+      sql("""select c5 from iud.dest150 where c1 = 'b' """),
+      Seq(Row("bbbz"))
+    )
+    sql("""drop table iud.dest150""")
+  }
+
+  test("update table with data for datatype mismatch with column ") {
+    sql("""update iud.update_01 set (imei) = ('skt') where level = 'aaa'""")
+    checkAnswer(
+      sql("""select * from iud.update_01 where imei = 'skt'"""),
+      Seq()
+    )
+  }
+
+  test("update carbon table-error[more columns in source table not allowed") {
+    val exception = intercept[Exception] {
+      sql("""update iud.dest d set (c2, c5 ) = (c2 + 1, concat(c5 , "z"), "abc")""").show()
+    }
+    assertResult("Number of source and destination columns are not matching")(exception.getMessage)
+  }
+
+  test("update carbon table-error[no set columns") {
+    intercept[Exception] {
+      sql("""update iud.dest d set () = ()""").show()
+    }
+  }
+
+  test("update carbon table-error[no set columns with updated column") {
+    intercept[Exception] {
+      sql("""update iud.dest d set  = (c1+1)""").show()
+    }
+  }
+  test("update carbon table-error[one set column with two updated column") {
+    intercept[Exception] {
+      sql("""update iud.dest  set c2 = (c2 + 1, concat(c5 , "z") )""").show()
+    }
+  }
+
+  test("""update carbon [special characters  in value- test parsing logic ]""") {
+    sql("""drop table if exists iud.dest160""")
+    sql("""create table iud.dest160 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest160""")
+    sql("""update iud.dest160 set(c1) = ("ab\')$*)(&^)")""").show()
+    sql("""update iud.dest160 set(c1) =  ('abd$asjdh$adasj$l;sdf$*)$*)(&^')""").show()
+    sql("""update iud.dest160 set(c1) =("\\")""").show()
+    sql("""update iud.dest160 set(c1) = ("ab\')$*)(&^)")""").show()
+    sql("""update iud.dest160 d set (c3,c5)=(select s.c33,'a\\a' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
+    sql("""update iud.dest160 d set (c3,c5)=(select s.c33,'\\' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
+    sql("""update iud.dest160 d set (c3,c5)=(select s.c33,'\\a' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
+    sql("""update iud.dest160 d set (c3,c5)      =     (select s.c33,'a\\a\\' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
+    sql("""update iud.dest160 d set (c3,c5) =(select s.c33,'a\'a\\' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
+    sql("""update iud.dest160 d set (c3,c5)=(select s.c33,'\\a\'a\"' from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
+    sql("""drop table iud.dest160""")
+  }
+
+  test("""update carbon [sub query, between and existing in outer condition.(Customer query ) ]""") {
+    sql("""drop table if exists iud.dest170""")
+    sql("""create table iud.dest170 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest170""")
+    sql("""update iud.dest170 d set (c3)=(select s.c33 from iud.source2 s where d.c1 = s.c11 and d.c2 = s.c22) where  d.c2 between 1 and 3""").show()
+    checkAnswer(
+      sql("""select c3 from  iud.dest170 as d where d.c2 between 1 and 3"""),
+      Seq(Row("MGM"), Row("RGK"), Row("cc"))
+    )
+    sql("""drop table iud.dest170""")
+  }
+
+  test("""update carbon [self join select query ]""") {
+    sql("""drop table if exists iud.dest171""")
+    sql("""create table iud.dest171 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest171""")
+    sql("""update iud.dest171 d set (c3)=(select concat(s.c3 , "z") from iud.dest171 s where d.c2 = s.c2)""").show
+    sql("""drop table if exists iud.dest172""")
+    sql("""create table iud.dest172 (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.dest172""")
+    sql("""update iud.dest172 d set (c3)=( concat(c3 , "z"))""").show
+    checkAnswer(
+      sql("""select c3 from  iud.dest171"""),
+      sql("""select c3 from  iud.dest172""")
+    )
+    sql("""drop table iud.dest171""")
+    sql("""drop table iud.dest172""")
+  }
+
+  test("update carbon table-error[closing bracket missed") {
+    intercept[Exception] {
+      sql("""update iud.dest d set (c2) = (194""").show()
+    }
+  }
+
+  test("update carbon table-error[starting bracket missed") {
+    intercept[Exception] {
+      sql("""update iud.dest d set (c2) = 194)""").show()
+    }
+  }
+
+  test("update carbon table-error[missing starting and closing bracket") {
+    intercept[Exception] {
+      sql("""update iud.dest d set (c2) = 194""").show()
+    }
+  }
+
+  test("test create table with column name as tupleID"){
+    intercept[Exception] {
+      sql("CREATE table carbontable (empno int, tupleID String, " +
+          "designation String, doj Timestamp, workgroupcategory int, " +
+          "workgroupcategoryname String, deptno int, deptname String, projectcode int, " +
+          "projectjoindate Timestamp, projectenddate Timestamp, attendance int, " +
+          "utilization int,salary int) STORED BY 'org.apache.carbondata.format' " +
+          "TBLPROPERTIES('DICTIONARY_INCLUDE'='empno,workgroupcategory,deptno,projectcode'," +
+          "'DICTIONARY_EXCLUDE'='empname')")
+    }
+  }
+
+  test("test show segment after updating data : JIRA-1411,JIRA-1414") {
+    sql("""drop table if exists iud.show_segment""").show
+    sql("""create table iud.show_segment (c1 string,c2 int,c3 string,c5 string) STORED BY 'org.apache.carbondata.format'""")
+    sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/dest.csv' INTO table iud.show_segment""")
+    val before_update = sql("""show segments for table iud.show_segment""").toDF()
+    sql("""update iud.show_segment d set (d.c3, d.c5 ) = (select s.c33,s.c55 from iud.source2 s where d.c1 = s.c11) where 1 = 1""").show()
+    val after_update = sql("""show segments for table iud.show_segment""").toDF()
+    checkAnswer(
+      before_update,
+      after_update
+    )
+    sql("""drop table if exists iud.show_segment""").show
+  }
+
+  test("Failure of update operation due to bad record with proper error message") {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FAIL")
+      val errorMessage = intercept[Exception] {
+        sql("drop table if exists update_with_bad_record")
+        sql("create table update_with_bad_record(item int, name String) stored by 'carbondata'")
+        sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/IUD/bad_record.csv' into table " +
+            s"update_with_bad_record")
+        sql("update update_with_bad_record set (item)=(3.45)").show()
+        sql("drop table if exists update_with_bad_record")
+      }
+      assert(errorMessage.getMessage.contains("Data load failed due to bad record"))
+    } finally {
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_BAD_RECORDS_ACTION, "FORCE")
+    }
+  }
+
+  test("More records after update operation ") {
+    sql("DROP TABLE IF EXISTS default.carbon1")
+    import sqlContext.implicits._
+    val df = sqlContext.sparkContext.parallelize(1 to 36000)
+      .map(x => (x+"a", "b", x))
+      .toDF("c1", "c2", "c3")
+    df.write
+      .format("carbondata")
+      .option("tableName", "carbon1")
+      .option("tempCSV", "true")
+      .option("compress", "true")
+      .mode(SaveMode.Overwrite)
+      .save()
+
+    checkAnswer(sql("select count(*) from default.carbon1"), Seq(Row(36000)))
+
+    sql("update default.carbon1 set (c1)=('test123') where c1='9999a'").show()
+
+    checkAnswer(sql("select count(*) from default.carbon1"), Seq(Row(36000)))
+
+    checkAnswer(sql("select * from default.carbon1 where c1 = 'test123'"), Row("test123","b",9999))
+
+    sql("DROP TABLE IF EXISTS default.carbon1")
+  }
 
   test("""CARBONDATA-1445 carbon.update.persist.enable=false it will fail to update data""") {
     CarbonProperties.getInstance()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
index 7790f59..4d5d39a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastore.scala
@@ -35,10 +35,10 @@ import org.apache.spark.sql.types._
 
 import org.apache.carbondata.common.logging.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.datastore.impl.FileFactory.FileType
-import org.apache.carbondata.core.locks.ZookeeperInit
-import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier}
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.reader.ThriftReader
@@ -383,6 +383,8 @@ class CarbonMetastore(hiveContext: HiveContext, val storePath: String,
       CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sqlContext)
       // discard cached table info in cachedDataSourceTables
       sqlContext.catalog.refreshTable(tableIdentifier)
+      DataMapStoreManager.getInstance().
+        clearDataMap(AbsoluteTableIdentifier.from(storePath, dbName, tableName))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/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 3d67a79..5335fe2 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
@@ -41,10 +41,11 @@ import org.apache.spark.util.SparkUtil
 import org.apache.carbondata.common.constants.LoggerAction
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.datastore.block.{Distributable, TableBlockInfo}
 import org.apache.carbondata.core.dictionary.server.DictionaryServer
 import org.apache.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnarFormatVersion}
 import org.apache.carbondata.core.metadata.datatype.DataType
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
@@ -438,6 +439,10 @@ object CarbonDataRDDFactory {
       threadArray.foreach {
         thread => thread.join()
       }
+      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getStorePath,
+        carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
+      val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
+      refresher.refreshSegments(validSegments.asJava)
     } catch {
       case e: Exception =>
         LOGGER.error(s"Exception when split partition: ${ e.getMessage }")
@@ -479,6 +484,10 @@ object CarbonDataRDDFactory {
       for (thread <- threadArray) {
         thread.join()
       }
+      val identifier = AbsoluteTableIdentifier.from(carbonLoadModel.getStorePath,
+        carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
+      val refresher = DataMapStoreManager.getInstance().getTableSegmentRefresher(identifier)
+      refresher.refreshSegments(validSegments.asJava)
     } catch {
       case e: Exception =>
         LOGGER.error(s"Exception when dropping partition: ${ e.getMessage }")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/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 4797884..75ad4ae 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
@@ -426,7 +426,7 @@ class CarbonFileMetastore extends CarbonMetaStore {
       CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
       // discard cached table info in cachedDataSourceTables
       sparkSession.sessionState.catalog.refreshTable(tableIdentifier)
-      DataMapStoreManager.getInstance().clearDataMap(identifier, "blocklet")
+      DataMapStoreManager.getInstance().clearDataMap(identifier)
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/eb771f5e/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 eb80b82..76241a6 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
@@ -83,7 +83,7 @@ class CarbonHiveMetaStore extends CarbonFileMetastore {
     CarbonHiveMetadataUtil.invalidateAndDropTable(dbName, tableName, sparkSession)
     // discard cached table info in cachedDataSourceTables
     sparkSession.sessionState.catalog.refreshTable(tableIdentifier)
-    DataMapStoreManager.getInstance().clearDataMap(identifier, "blocklet")
+    DataMapStoreManager.getInstance().clearDataMap(identifier)
   }
 
   override def checkSchemasModifiedTimeAndReloadTables(storePath: String) {


[28/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java
deleted file mode 100644
index 683696e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/TimSort.java
+++ /dev/null
@@ -1,986 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.sort.unsafe.sort;
-
-import java.util.Comparator;
-
-import org.apache.spark.util.collection.SortDataFormat;
-
-/**
- * A port of the Apache Spark's TimSort and they originally ported from Android TimSort class,
- * which utilizes a "stable, adaptive, iterative mergesort."
- * See the method comment on sort() for more details.
- *
- * This has been kept in Java with the original style in order to match very closely with the
- * Android source code, and thus be easy to verify correctness. The class is package private. We put
- * a simple Scala wrapper {@link org.apache.spark.util.collection.Sorter}, which is available to
- * package org.apache.spark.
- *
- * The purpose of the port is to generalize the interface to the sort to accept input data formats
- * besides simple arrays where every element is sorted individually. For instance, the AppendOnlyMap
- * uses this to sort an Array with alternating elements of the form [key, value, key, value].
- * This generalization comes with minimal overhead -- see SortDataFormat for more information.
- *
- * We allow key reuse to prevent creating many key objects -- see SortDataFormat.
- *
- * @see SortDataFormat
- * @see org.apache.spark.util.collection.Sorter
- */
-public class TimSort<K, Buffer> {
-
-  /**
-   * This is the minimum sized sequence that will be merged.  Shorter
-   * sequences will be lengthened by calling binarySort.  If the entire
-   * array is less than this length, no merges will be performed.
-   *
-   * This constant should be a power of two.  It was 64 in Tim Peter's C
-   * implementation, but 32 was empirically determined to work better in
-   * this implementation.  In the unlikely event that you set this constant
-   * to be a number that's not a power of two, you'll need to change the
-   * minRunLength computation.
-   *
-   * If you decrease this constant, you must change the stackLen
-   * computation in the TimSort constructor, or you risk an
-   * ArrayOutOfBounds exception.  See listsort.txt for a discussion
-   * of the minimum stack length required as a function of the length
-   * of the array being sorted and the minimum merge sequence length.
-   */
-  private static final int MIN_MERGE = 32;
-
-  private final SortDataFormat<K, Buffer> s;
-
-  public TimSort(SortDataFormat<K, Buffer> sortDataFormat) {
-    this.s = sortDataFormat;
-  }
-
-  /**
-   * A stable, adaptive, iterative mergesort that requires far fewer than
-   * n lg(n) comparisons when running on partially sorted arrays, while
-   * offering performance comparable to a traditional mergesort when run
-   * on random arrays.  Like all proper mergesorts, this sort is stable and
-   * runs O(n log n) time (worst case).  In the worst case, this sort requires
-   * temporary storage space for n/2 object references; in the best case,
-   * it requires only a small constant amount of space.
-   *
-   * This implementation was adapted from Tim Peters's list sort for
-   * Python, which is described in detail here:
-   *
-   *   http://svn.python.org/projects/python/trunk/Objects/listsort.txt
-   *
-   * Tim's C code may be found here:
-   *
-   *   http://svn.python.org/projects/python/trunk/Objects/listobject.c
-   *
-   * The underlying techniques are described in this paper (and may have
-   * even earlier origins):
-   *
-   *  "Optimistic Sorting and Information Theoretic Complexity"
-   *  Peter McIlroy
-   *  SODA (Fourth Annual ACM-SIAM Symposium on Discrete Algorithms),
-   *  pp 467-474, Austin, Texas, 25-27 January 1993.
-   *
-   * While the API to this class consists solely of static methods, it is
-   * (privately) instantiable; a TimSort instance holds the state of an ongoing
-   * sort, assuming the input array is large enough to warrant the full-blown
-   * TimSort. Small arrays are sorted in place, using a binary insertion sort.
-   */
-  public void sort(Buffer a, int lo, int hi, Comparator<? super K> c) {
-    assert c != null;
-
-    int nRemaining  = hi - lo;
-    if (nRemaining < 2) {
-      return;  // Arrays of size 0 and 1 are always sorted
-    }
-
-    // If array is small, do a "mini-TimSort" with no merges
-    if (nRemaining < MIN_MERGE) {
-      int initRunLen = countRunAndMakeAscending(a, lo, hi, c);
-      binarySort(a, lo, hi, lo + initRunLen, c);
-      return;
-    }
-
-    /*
-     * March over the array once, left to right, finding natural runs,
-     * extending short natural runs to minRun elements, and merging runs
-     * to maintain stack invariant.
-     */
-    SortState sortState = new SortState(a, c, hi - lo);
-    int minRun = minRunLength(nRemaining);
-    do {
-      // Identify next run
-      int runLen = countRunAndMakeAscending(a, lo, hi, c);
-
-      // If run is short, extend to min(minRun, nRemaining)
-      if (runLen < minRun) {
-        int force = nRemaining <= minRun ? nRemaining : minRun;
-        binarySort(a, lo, lo + force, lo + runLen, c);
-        runLen = force;
-      }
-
-      // Push run onto pending-run stack, and maybe merge
-      sortState.pushRun(lo, runLen);
-      sortState.mergeCollapse();
-
-      // Advance to find next run
-      lo += runLen;
-      nRemaining -= runLen;
-    } while (nRemaining != 0);
-
-    // Merge all remaining runs to complete sort
-    assert lo == hi;
-    sortState.mergeForceCollapse();
-    assert sortState.stackSize == 1;
-  }
-
-  /**
-   * Sorts the specified portion of the specified array using a binary
-   * insertion sort.  This is the best method for sorting small numbers
-   * of elements.  It requires O(n log n) compares, but O(n^2) data
-   * movement (worst case).
-   *
-   * If the initial part of the specified range is already sorted,
-   * this method can take advantage of it: the method assumes that the
-   * elements from index {@code lo}, inclusive, to {@code start},
-   * exclusive are already sorted.
-   *
-   * @param a the array in which a range is to be sorted
-   * @param lo the index of the first element in the range to be sorted
-   * @param hi the index after the last element in the range to be sorted
-   * @param start the index of the first element in the range that is
-   *        not already known to be sorted ({@code lo <= start <= hi})
-   * @param c comparator to used for the sort
-   */
-  @SuppressWarnings("fallthrough")
-  private void binarySort(Buffer a, int lo, int hi, int start, Comparator<? super K> c) {
-    assert lo <= start && start <= hi;
-    if (start == lo) {
-      start++;
-    }
-
-    K key0 = s.newKey();
-    K key1 = s.newKey();
-
-    Buffer pivotStore = s.allocate(1);
-    for ( ; start < hi; start++) {
-      s.copyElement(a, start, pivotStore, 0);
-      K pivot = s.getKey(pivotStore, 0, key0);
-
-      // Set left (and right) to the index where a[start] (pivot) belongs
-      int left = lo;
-      int right = start;
-      assert left <= right;
-      /*
-       * Invariants:
-       *   pivot >= all in [lo, left).
-       *   pivot <  all in [right, start).
-       */
-      while (left < right) {
-        int mid = (left + right) >>> 1;
-        if (c.compare(pivot, s.getKey(a, mid, key1)) < 0) {
-          right = mid;
-        }
-        else {
-          left = mid + 1;
-        }
-      }
-      assert left == right;
-
-      /*
-       * The invariants still hold: pivot >= all in [lo, left) and
-       * pivot < all in [left, start), so pivot belongs at left.  Note
-       * that if there are elements equal to pivot, left points to the
-       * first slot after them -- that's why this sort is stable.
-       * Slide elements over to make room for pivot.
-       */
-      int n = start - left;  // The number of elements to move
-      // Switch is just an optimization for arraycopy in default case
-      switch (n) {
-        case 2:  {
-          s.copyElement(a, left + 1, a, left + 2);
-          s.copyElement(a, left, a, left + 1);
-          break;
-        }
-        case 1:  {
-          s.copyElement(a, left, a, left + 1);
-          break;
-        }
-        default: s.copyRange(a, left, a, left + 1, n);
-      }
-      s.copyElement(pivotStore, 0, a, left);
-    }
-  }
-
-  /**
-   * Returns the length of the run beginning at the specified position in
-   * the specified array and reverses the run if it is descending (ensuring
-   * that the run will always be ascending when the method returns).
-   *
-   * A run is the longest ascending sequence with:
-   *
-   *    a[lo] <= a[lo + 1] <= a[lo + 2] <= ...
-   *
-   * or the longest descending sequence with:
-   *
-   *    a[lo] >  a[lo + 1] >  a[lo + 2] >  ...
-   *
-   * For its intended use in a stable mergesort, the strictness of the
-   * definition of "descending" is needed so that the call can safely
-   * reverse a descending sequence without violating stability.
-   *
-   * @param a the array in which a run is to be counted and possibly reversed
-   * @param lo index of the first element in the run
-   * @param hi index after the last element that may be contained in the run.
-  It is required that {@code lo < hi}.
-   * @param c the comparator to used for the sort
-   * @return  the length of the run beginning at the specified position in
-   *          the specified array
-   */
-  private int countRunAndMakeAscending(Buffer a, int lo, int hi, Comparator<? super K> c) {
-    assert lo < hi;
-    int runHi = lo + 1;
-    if (runHi == hi) {
-      return 1;
-    }
-
-    K key0 = s.newKey();
-    K key1 = s.newKey();
-
-    // Find end of run, and reverse range if descending
-    if (c.compare(s.getKey(a, runHi++, key0), s.getKey(a, lo, key1)) < 0) { // Descending
-      while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) < 0) {
-        runHi++;
-      }
-      reverseRange(a, lo, runHi);
-    } else {                              // Ascending
-      while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) >= 0) {
-        runHi++;
-      }
-    }
-
-    return runHi - lo;
-  }
-
-  /**
-   * Reverse the specified range of the specified array.
-   *
-   * @param a the array in which a range is to be reversed
-   * @param lo the index of the first element in the range to be reversed
-   * @param hi the index after the last element in the range to be reversed
-   */
-  private void reverseRange(Buffer a, int lo, int hi) {
-    hi--;
-    while (lo < hi) {
-      s.swap(a, lo, hi);
-      lo++;
-      hi--;
-    }
-  }
-
-  /**
-   * Returns the minimum acceptable run length for an array of the specified
-   * length. Natural runs shorter than this will be extended with
-   * {@link #binarySort}.
-   *
-   * Roughly speaking, the computation is:
-   *
-   *  If n < MIN_MERGE, return n (it's too small to bother with fancy stuff).
-   *  Else if n is an exact power of 2, return MIN_MERGE/2.
-   *  Else return an int k, MIN_MERGE/2 <= k <= MIN_MERGE, such that n/k
-   *   is close to, but strictly less than, an exact power of 2.
-   *
-   * For the rationale, see listsort.txt.
-   *
-   * @param n the length of the array to be sorted
-   * @return the length of the minimum run to be merged
-   */
-  private int minRunLength(int n) {
-    assert n >= 0;
-    int r = 0;      // Becomes 1 if any 1 bits are shifted off
-    while (n >= MIN_MERGE) {
-      r |= (n & 1);
-      n >>= 1;
-    }
-    return n + r;
-  }
-
-  private class SortState {
-
-    /**
-     * The Buffer being sorted.
-     */
-    private final Buffer a;
-
-    /**
-     * Length of the sort Buffer.
-     */
-    private final int aLength;
-
-    /**
-     * The comparator for this sort.
-     */
-    private final Comparator<? super K> c;
-
-    /**
-     * When we get into galloping mode, we stay there until both runs win less
-     * often than MIN_GALLOP consecutive times.
-     */
-    private static final int  MIN_GALLOP = 7;
-
-    /**
-     * This controls when we get *into* galloping mode.  It is initialized
-     * to MIN_GALLOP.  The mergeLo and mergeHi methods nudge it higher for
-     * random data, and lower for highly structured data.
-     */
-    private int minGallop = MIN_GALLOP;
-
-    /**
-     * Maximum initial size of tmp array, which is used for merging.  The array
-     * can grow to accommodate demand.
-     *
-     * Unlike Tim's original C version, we do not allocate this much storage
-     * when sorting smaller arrays.  This change was required for performance.
-     */
-    private static final int INITIAL_TMP_STORAGE_LENGTH = 256;
-
-    /**
-     * Temp storage for merges.
-     */
-    private Buffer tmp; // Actual runtime type will be Object[], regardless of T
-
-    /**
-     * Length of the temp storage.
-     */
-    private int tmpLength = 0;
-
-    /**
-     * A stack of pending runs yet to be merged.  Run i starts at
-     * address base[i] and extends for len[i] elements.  It's always
-     * true (so long as the indices are in bounds) that:
-     *
-     *     runBase[i] + runLen[i] == runBase[i + 1]
-     *
-     * so we could cut the storage for this, but it's a minor amount,
-     * and keeping all the info explicit simplifies the code.
-     */
-    private int stackSize = 0;  // Number of pending runs on stack
-    private final int[] runBase;
-    private final int[] runLen;
-
-    /**
-     * Creates a TimSort instance to maintain the state of an ongoing sort.
-     *
-     * @param a the array to be sorted
-     * @param c the comparator to determine the order of the sort
-     */
-    private SortState(Buffer a, Comparator<? super K> c, int len) {
-      this.aLength = len;
-      this.a = a;
-      this.c = c;
-
-      // Allocate temp storage (which may be increased later if necessary)
-      tmpLength = len < 2 * INITIAL_TMP_STORAGE_LENGTH ? len >>> 1 : INITIAL_TMP_STORAGE_LENGTH;
-      tmp = s.allocate(tmpLength);
-
-      /*
-       * Allocate runs-to-be-merged stack (which cannot be expanded).  The
-       * stack length requirements are described in listsort.txt.  The C
-       * version always uses the same stack length (85), but this was
-       * measured to be too expensive when sorting "mid-sized" arrays (e.g.,
-       * 100 elements) in Java.  Therefore, we use smaller (but sufficiently
-       * large) stack lengths for smaller arrays.  The "magic numbers" in the
-       * computation below must be changed if MIN_MERGE is decreased.  See
-       * the MIN_MERGE declaration above for more information.
-       */
-      int stackLen = (len <    120  ?  5 :
-                      len <   1542  ? 10 :
-                      len < 119151  ? 19 : 40);
-      runBase = new int[stackLen];
-      runLen = new int[stackLen];
-    }
-
-    /**
-     * Pushes the specified run onto the pending-run stack.
-     *
-     * @param runBase index of the first element in the run
-     * @param runLen  the number of elements in the run
-     */
-    private void pushRun(int runBase, int runLen) {
-      this.runBase[stackSize] = runBase;
-      this.runLen[stackSize] = runLen;
-      stackSize++;
-    }
-
-    /**
-     * Examines the stack of runs waiting to be merged and merges adjacent runs
-     * until the stack invariants are reestablished:
-     *
-     *     1. runLen[i - 3] > runLen[i - 2] + runLen[i - 1]
-     *     2. runLen[i - 2] > runLen[i - 1]
-     *
-     * This method is called each time a new run is pushed onto the stack,
-     * so the invariants are guaranteed to hold for i < stackSize upon
-     * entry to the method.
-     */
-    private void mergeCollapse() {
-      while (stackSize > 1) {
-        int n = stackSize - 2;
-        if ((n >= 1 && runLen[n - 1] <= runLen[n] + runLen[n + 1])
-            || (n >= 2 && runLen[n - 2] <= runLen[n] + runLen[n - 1])) {
-          if (runLen[n - 1] < runLen[n + 1]) {
-            n--;
-          }
-        } else if (runLen[n] > runLen[n + 1]) {
-          break; // Invariant is established
-        }
-        mergeAt(n);
-      }
-    }
-
-    /**
-     * Merges all runs on the stack until only one remains.  This method is
-     * called once, to complete the sort.
-     */
-    private void mergeForceCollapse() {
-      while (stackSize > 1) {
-        int n = stackSize - 2;
-        if (n > 0 && runLen[n - 1] < runLen[n + 1]) {
-          n--;
-        }
-        mergeAt(n);
-      }
-    }
-
-    /**
-     * Merges the two runs at stack indices i and i+1.  Run i must be
-     * the penultimate or antepenultimate run on the stack.  In other words,
-     * i must be equal to stackSize-2 or stackSize-3.
-     *
-     * @param i stack index of the first of the two runs to merge
-     */
-    private void mergeAt(int i) {
-      assert stackSize >= 2;
-      assert i >= 0;
-      assert i == stackSize - 2 || i == stackSize - 3;
-
-      int base1 = runBase[i];
-      int len1 = runLen[i];
-      int base2 = runBase[i + 1];
-      int len2 = runLen[i + 1];
-      assert len1 > 0 && len2 > 0;
-      assert base1 + len1 == base2;
-
-      /*
-       * Record the length of the combined runs; if i is the 3rd-last
-       * run now, also slide over the last run (which isn't involved
-       * in this merge).  The current run (i+1) goes away in any case.
-       */
-      runLen[i] = len1 + len2;
-      if (i == stackSize - 3) {
-        runBase[i + 1] = runBase[i + 2];
-        runLen[i + 1] = runLen[i + 2];
-      }
-      stackSize--;
-
-      K key0 = s.newKey();
-
-      /*
-       * Find where the first element of run2 goes in run1. Prior elements
-       * in run1 can be ignored (because they're already in place).
-       */
-      int k = gallopRight(s.getKey(a, base2, key0), a, base1, len1, 0, c);
-      assert k >= 0;
-      base1 += k;
-      len1 -= k;
-      if (len1 == 0) {
-        return;
-      }
-
-      /*
-       * Find where the last element of run1 goes in run2. Subsequent elements
-       * in run2 can be ignored (because they're already in place).
-       */
-      len2 = gallopLeft(s.getKey(a, base1 + len1 - 1, key0), a, base2, len2, len2 - 1, c);
-      assert len2 >= 0;
-      if (len2 == 0) {
-        return;
-      }
-
-      // Merge remaining runs, using tmp array with min(len1, len2) elements
-      if (len1 <= len2) {
-        mergeLo(base1, len1, base2, len2);
-      }
-      else {
-        mergeHi(base1, len1, base2, len2);
-      }
-    }
-
-    /**
-     * Locates the position at which to insert the specified key into the
-     * specified sorted range; if the range contains an element equal to key,
-     * returns the index of the leftmost equal element.
-     *
-     * @param key the key whose insertion point to search for
-     * @param a the array in which to search
-     * @param base the index of the first element in the range
-     * @param len the length of the range; must be > 0
-     * @param hint the index at which to begin the search, 0 <= hint < n.
-     *     The closer hint is to the result, the faster this method will run.
-     * @param c the comparator used to order the range, and to search
-     * @return the int k,  0 <= k <= n such that a[b + k - 1] < key <= a[b + k],
-     *    pretending that a[b - 1] is minus infinity and a[b + n] is infinity.
-     *    In other words, key belongs at index b + k; or in other words,
-     *    the first k elements of a should precede key, and the last n - k
-     *    should follow it.
-     */
-    private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator<? super K> c) {
-      assert len > 0 && hint >= 0 && hint < len;
-      int lastOfs = 0;
-      int ofs = 1;
-      K key0 = s.newKey();
-
-      if (c.compare(key, s.getKey(a, base + hint, key0)) > 0) {
-        // Gallop right until a[base+hint+lastOfs] < key <= a[base+hint+ofs]
-        int maxOfs = len - hint;
-        while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key0)) > 0) {
-          lastOfs = ofs;
-          ofs = (ofs << 1) + 1;
-          if (ofs <= 0) {  // int overflow
-            ofs = maxOfs;
-          }
-        }
-        if (ofs > maxOfs) {
-          ofs = maxOfs;
-        }
-
-        // Make offsets relative to base
-        lastOfs += hint;
-        ofs += hint;
-      } else { // key <= a[base + hint]
-        // Gallop left until a[base+hint-ofs] < key <= a[base+hint-lastOfs]
-        final int maxOfs = hint + 1;
-        while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key0)) <= 0) {
-          lastOfs = ofs;
-          ofs = (ofs << 1) + 1;
-          if (ofs <= 0) {  // int overflow
-            ofs = maxOfs;
-          }
-        }
-        if (ofs > maxOfs) {
-          ofs = maxOfs;
-        }
-
-        // Make offsets relative to base
-        int tmp = lastOfs;
-        lastOfs = hint - ofs;
-        ofs = hint - tmp;
-      }
-      assert -1 <= lastOfs && lastOfs < ofs && ofs <= len;
-
-      /*
-       * Now a[base+lastOfs] < key <= a[base+ofs], so key belongs somewhere
-       * to the right of lastOfs but no farther right than ofs.  Do a binary
-       * search, with invariant a[base + lastOfs - 1] < key <= a[base + ofs].
-       */
-      lastOfs++;
-      while (lastOfs < ofs) {
-        int m = lastOfs + ((ofs - lastOfs) >>> 1);
-
-        if (c.compare(key, s.getKey(a, base + m, key0)) > 0) {
-          lastOfs = m + 1;  // a[base + m] < key
-        }
-        else {
-          ofs = m;          // key <= a[base + m]
-        }
-      }
-      assert lastOfs == ofs;    // so a[base + ofs - 1] < key <= a[base + ofs]
-      return ofs;
-    }
-
-    /**
-     * Like gallopLeft, except that if the range contains an element equal to
-     * key, gallopRight returns the index after the rightmost equal element.
-     *
-     * @param key the key whose insertion point to search for
-     * @param a the array in which to search
-     * @param base the index of the first element in the range
-     * @param len the length of the range; must be > 0
-     * @param hint the index at which to begin the search, 0 <= hint < n.
-     *     The closer hint is to the result, the faster this method will run.
-     * @param c the comparator used to order the range, and to search
-     * @return the int k,  0 <= k <= n such that a[b + k - 1] <= key < a[b + k]
-     */
-    private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator<? super K> c) {
-      assert len > 0 && hint >= 0 && hint < len;
-
-      int ofs = 1;
-      int lastOfs = 0;
-      K key1 = s.newKey();
-
-      if (c.compare(key, s.getKey(a, base + hint, key1)) < 0) {
-        // Gallop left until a[b+hint - ofs] <= key < a[b+hint - lastOfs]
-        int maxOfs = hint + 1;
-        while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key1)) < 0) {
-          lastOfs = ofs;
-          ofs = (ofs << 1) + 1;
-          if (ofs <= 0) {  // int overflow
-            ofs = maxOfs;
-          }
-        }
-        if (ofs > maxOfs) {
-          ofs = maxOfs;
-        }
-
-        // Make offsets relative to b
-        int tmp = lastOfs;
-        lastOfs = hint - ofs;
-        ofs = hint - tmp;
-      } else { // a[b + hint] <= key
-        // Gallop right until a[b+hint + lastOfs] <= key < a[b+hint + ofs]
-        int maxOfs = len - hint;
-        while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key1)) >= 0) {
-          lastOfs = ofs;
-          ofs = (ofs << 1) + 1;
-          if (ofs <= 0) {  // int overflow
-            ofs = maxOfs;
-          }
-        }
-        if (ofs > maxOfs) {
-          ofs = maxOfs;
-        }
-
-        // Make offsets relative to b
-        lastOfs += hint;
-        ofs += hint;
-      }
-      assert -1 <= lastOfs && lastOfs < ofs && ofs <= len;
-
-      /*
-       * Now a[b + lastOfs] <= key < a[b + ofs], so key belongs somewhere to
-       * the right of lastOfs but no farther right than ofs.  Do a binary
-       * search, with invariant a[b + lastOfs - 1] <= key < a[b + ofs].
-       */
-      lastOfs++;
-      while (lastOfs < ofs) {
-        int m = lastOfs + ((ofs - lastOfs) >>> 1);
-
-        if (c.compare(key, s.getKey(a, base + m, key1)) < 0) {
-          ofs = m;          // key < a[b + m]
-        }
-        else {
-          lastOfs = m + 1;  // a[b + m] <= key
-        }
-      }
-      assert lastOfs == ofs;    // so a[b + ofs - 1] <= key < a[b + ofs]
-      return ofs;
-    }
-
-    /**
-     * Merges two adjacent runs in place, in a stable fashion.  The first
-     * element of the first run must be greater than the first element of the
-     * second run (a[base1] > a[base2]), and the last element of the first run
-     * (a[base1 + len1-1]) must be greater than all elements of the second run.
-     *
-     * For performance, this method should be called only when len1 <= len2;
-     * its twin, mergeHi should be called if len1 >= len2.  (Either method
-     * may be called if len1 == len2.)
-     *
-     * @param base1 index of first element in first run to be merged
-     * @param len1  length of first run to be merged (must be > 0)
-     * @param base2 index of first element in second run to be merged
-     *        (must be aBase + aLen)
-     * @param len2  length of second run to be merged (must be > 0)
-     */
-    private void mergeLo(int base1, int len1, int base2, int len2) {
-      assert len1 > 0 && len2 > 0 && base1 + len1 == base2;
-
-      // Copy first run into temp array
-      Buffer a = this.a; // For performance
-      Buffer tmp = ensureCapacity(len1);
-      s.copyRange(a, base1, tmp, 0, len1);
-
-      int cursor1 = 0;       // Indexes into tmp array
-      int cursor2 = base2;   // Indexes int a
-      int dest = base1;      // Indexes int a
-
-      // Move first element of second run and deal with degenerate cases
-      s.copyElement(a, cursor2++, a, dest++);
-      if (--len2 == 0) {
-        s.copyRange(tmp, cursor1, a, dest, len1);
-        return;
-      }
-      if (len1 == 1) {
-        s.copyRange(a, cursor2, a, dest, len2);
-        s.copyElement(tmp, cursor1, a, dest + len2); // Last elt of run 1 to end of merge
-        return;
-      }
-
-      K key0 = s.newKey();
-      K key1 = s.newKey();
-
-      Comparator<? super K> c = this.c;  // Use local variable for performance
-      int minGallop = this.minGallop;    //  "    "       "     "      "
-      outer:
-      while (true) {
-        int count1 = 0; // Number of times in a row that first run won
-        int count2 = 0; // Number of times in a row that second run won
-
-        /*
-         * Do the straightforward thing until (if ever) one run starts
-         * winning consistently.
-         */
-        do {
-          assert len1 > 1 && len2 > 0;
-          if (c.compare(s.getKey(a, cursor2, key0), s.getKey(tmp, cursor1, key1)) < 0) {
-            s.copyElement(a, cursor2++, a, dest++);
-            count2++;
-            count1 = 0;
-            if (--len2 == 0) {
-              break outer;
-            }
-          } else {
-            s.copyElement(tmp, cursor1++, a, dest++);
-            count1++;
-            count2 = 0;
-            if (--len1 == 1) {
-              break outer;
-            }
-          }
-        } while ((count1 | count2) < minGallop);
-
-        /*
-         * One run is winning so consistently that galloping may be a
-         * huge win. So try that, and continue galloping until (if ever)
-         * neither run appears to be winning consistently anymore.
-         */
-        do {
-          assert len1 > 1 && len2 > 0;
-          count1 = gallopRight(s.getKey(a, cursor2, key0), tmp, cursor1, len1, 0, c);
-          if (count1 != 0) {
-            s.copyRange(tmp, cursor1, a, dest, count1);
-            dest += count1;
-            cursor1 += count1;
-            len1 -= count1;
-            if (len1 <= 1) { // len1 == 1 || len1 == 0
-              break outer;
-            }
-          }
-          s.copyElement(a, cursor2++, a, dest++);
-          if (--len2 == 0) {
-            break outer;
-          }
-
-          count2 = gallopLeft(s.getKey(tmp, cursor1, key0), a, cursor2, len2, 0, c);
-          if (count2 != 0) {
-            s.copyRange(a, cursor2, a, dest, count2);
-            dest += count2;
-            cursor2 += count2;
-            len2 -= count2;
-            if (len2 == 0) {
-              break outer;
-            }
-          }
-          s.copyElement(tmp, cursor1++, a, dest++);
-          if (--len1 == 1) {
-            break outer;
-          }
-          minGallop--;
-        } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP);
-        if (minGallop < 0) {
-          minGallop = 0;
-        }
-        minGallop += 2;  // Penalize for leaving gallop mode
-      }  // End of "outer" loop
-      this.minGallop = minGallop < 1 ? 1 : minGallop;  // Write back to field
-
-      if (len1 == 1) {
-        assert len2 > 0;
-        s.copyRange(a, cursor2, a, dest, len2);
-        s.copyElement(tmp, cursor1, a, dest + len2); //  Last elt of run 1 to end of merge
-      } else if (len1 == 0) {
-        throw new IllegalArgumentException(
-            "Comparison method violates its general contract!");
-      } else {
-        assert len2 == 0;
-        assert len1 > 1;
-        s.copyRange(tmp, cursor1, a, dest, len1);
-      }
-    }
-
-    /**
-     * Like mergeLo, except that this method should be called only if
-     * len1 >= len2; mergeLo should be called if len1 <= len2.  (Either method
-     * may be called if len1 == len2.)
-     *
-     * @param base1 index of first element in first run to be merged
-     * @param len1  length of first run to be merged (must be > 0)
-     * @param base2 index of first element in second run to be merged
-     *        (must be aBase + aLen)
-     * @param len2  length of second run to be merged (must be > 0)
-     */
-    private void mergeHi(int base1, int len1, int base2, int len2) {
-      assert len1 > 0 && len2 > 0 && base1 + len1 == base2;
-
-      // Copy second run into temp array
-      Buffer a = this.a; // For performance
-      Buffer tmp = ensureCapacity(len2);
-      s.copyRange(a, base2, tmp, 0, len2);
-
-      int cursor1 = base1 + len1 - 1;  // Indexes into a
-      int cursor2 = len2 - 1;          // Indexes into tmp array
-      int dest = base2 + len2 - 1;     // Indexes into a
-
-      K key0 = s.newKey();
-      K key1 = s.newKey();
-
-      // Move last element of first run and deal with degenerate cases
-      s.copyElement(a, cursor1--, a, dest--);
-      if (--len1 == 0) {
-        s.copyRange(tmp, 0, a, dest - (len2 - 1), len2);
-        return;
-      }
-      if (len2 == 1) {
-        dest -= len1;
-        cursor1 -= len1;
-        s.copyRange(a, cursor1 + 1, a, dest + 1, len1);
-        s.copyElement(tmp, cursor2, a, dest);
-        return;
-      }
-
-      Comparator<? super K> c = this.c;  // Use local variable for performance
-      int minGallop = this.minGallop;    //  "    "       "     "      "
-      outer:
-      while (true) {
-        int count1 = 0; // Number of times in a row that first run won
-        int count2 = 0; // Number of times in a row that second run won
-
-        /*
-         * Do the straightforward thing until (if ever) one run
-         * appears to win consistently.
-         */
-        do {
-          assert len1 > 0 && len2 > 1;
-          if (c.compare(s.getKey(tmp, cursor2, key0), s.getKey(a, cursor1, key1)) < 0) {
-            s.copyElement(a, cursor1--, a, dest--);
-            count1++;
-            count2 = 0;
-            if (--len1 == 0) {
-              break outer;
-            }
-          } else {
-            s.copyElement(tmp, cursor2--, a, dest--);
-            count2++;
-            count1 = 0;
-            if (--len2 == 1) {
-              break outer;
-            }
-          }
-        } while ((count1 | count2) < minGallop);
-
-        /*
-         * One run is winning so consistently that galloping may be a
-         * huge win. So try that, and continue galloping until (if ever)
-         * neither run appears to be winning consistently anymore.
-         */
-        do {
-          assert len1 > 0 && len2 > 1;
-          count1 = len1 - gallopRight(s.getKey(tmp, cursor2, key0), a, base1, len1, len1 - 1, c);
-          if (count1 != 0) {
-            dest -= count1;
-            cursor1 -= count1;
-            len1 -= count1;
-            s.copyRange(a, cursor1 + 1, a, dest + 1, count1);
-            if (len1 == 0) {
-              break outer;
-            }
-          }
-          s.copyElement(tmp, cursor2--, a, dest--);
-          if (--len2 == 1) {
-            break outer;
-          }
-
-          count2 = len2 - gallopLeft(s.getKey(a, cursor1, key0), tmp, 0, len2, len2 - 1, c);
-          if (count2 != 0) {
-            dest -= count2;
-            cursor2 -= count2;
-            len2 -= count2;
-            s.copyRange(tmp, cursor2 + 1, a, dest + 1, count2);
-            if (len2 <= 1) { // len2 == 1 || len2 == 0
-              break outer;
-            }
-          }
-          s.copyElement(a, cursor1--, a, dest--);
-          if (--len1 == 0) {
-            break outer;
-          }
-          minGallop--;
-        } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP);
-        if (minGallop < 0) {
-          minGallop = 0;
-        }
-        minGallop += 2;  // Penalize for leaving gallop mode
-      }  // End of "outer" loop
-      this.minGallop = minGallop < 1 ? 1 : minGallop;  // Write back to field
-
-      if (len2 == 1) {
-        assert len1 > 0;
-        dest -= len1;
-        cursor1 -= len1;
-        s.copyRange(a, cursor1 + 1, a, dest + 1, len1);
-        s.copyElement(tmp, cursor2, a, dest); // Move first elt of run2 to front of merge
-      } else if (len2 == 0) {
-        throw new IllegalArgumentException(
-            "Comparison method violates its general contract!");
-      } else {
-        assert len1 == 0;
-        assert len2 > 0;
-        s.copyRange(tmp, 0, a, dest - (len2 - 1), len2);
-      }
-    }
-
-    /**
-     * Ensures that the external array tmp has at least the specified
-     * number of elements, increasing its size if necessary.  The size
-     * increases exponentially to ensure amortized linear time complexity.
-     *
-     * @param minCapacity the minimum required capacity of the tmp array
-     * @return tmp, whether or not it grew
-     */
-    private Buffer ensureCapacity(int minCapacity) {
-      if (tmpLength < minCapacity) {
-        // Compute smallest power of 2 > minCapacity
-        int newSize = minCapacity;
-        newSize |= newSize >> 1;
-        newSize |= newSize >> 2;
-        newSize |= newSize >> 4;
-        newSize |= newSize >> 8;
-        newSize |= newSize >> 16;
-        newSize++;
-
-        if (newSize < 0) { // Not bloody likely!
-          newSize = minCapacity;
-        }
-        else {
-          newSize = Math.min(newSize, aLength >>> 1);
-        }
-
-        tmp = s.allocate(newSize);
-        tmpLength = newSize;
-      }
-      return tmp;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/UnsafeIntSortDataFormat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/UnsafeIntSortDataFormat.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/UnsafeIntSortDataFormat.java
deleted file mode 100644
index e7fec26..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/unsafe/sort/UnsafeIntSortDataFormat.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.sort.unsafe.sort;
-
-import org.apache.carbondata.core.memory.IntPointerBuffer;
-import org.apache.carbondata.processing.newflow.sort.unsafe.UnsafeCarbonRowPage;
-import org.apache.carbondata.processing.newflow.sort.unsafe.holder.UnsafeCarbonRow;
-
-import org.apache.spark.util.collection.SortDataFormat;
-
-/**
- * Interface implementation for utilities to sort the data.
- */
-public class UnsafeIntSortDataFormat
-    extends SortDataFormat<UnsafeCarbonRow, IntPointerBuffer> {
-
-  private UnsafeCarbonRowPage page;
-
-  public UnsafeIntSortDataFormat(UnsafeCarbonRowPage page) {
-    this.page = page;
-  }
-
-  @Override public UnsafeCarbonRow getKey(IntPointerBuffer data, int pos) {
-    // Since we re-use keys, this method shouldn't be called.
-    throw new UnsupportedOperationException();
-  }
-
-  @Override public UnsafeCarbonRow newKey() {
-    return new UnsafeCarbonRow();
-  }
-
-  @Override
-  public UnsafeCarbonRow getKey(IntPointerBuffer data, int pos, UnsafeCarbonRow reuse) {
-    reuse.address = data.get(pos) + page.getDataBlock().getBaseOffset();
-    return reuse;
-  }
-
-  @Override public void swap(IntPointerBuffer data, int pos0, int pos1) {
-    int tempPointer = data.get(pos0);
-    data.set(pos0, data.get(pos1));
-    data.set(pos1, tempPointer);
-  }
-
-  @Override
-  public void copyElement(IntPointerBuffer src, int srcPos, IntPointerBuffer dst, int dstPos) {
-    dst.set(dstPos, src.get(srcPos));
-  }
-
-  @Override
-  public void copyRange(IntPointerBuffer src, int srcPos, IntPointerBuffer dst, int dstPos,
-      int length) {
-    System.arraycopy(src.getPointerBlock(), srcPos, dst.getPointerBlock(), dstPos, length);
-  }
-
-  @Override public IntPointerBuffer allocate(int length) {
-    return new IntPointerBuffer(length);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/CarbonRowDataWriterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/CarbonRowDataWriterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/CarbonRowDataWriterProcessorStepImpl.java
deleted file mode 100644
index 643d2ba..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/CarbonRowDataWriterProcessorStepImpl.java
+++ /dev/null
@@ -1,299 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.steps;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-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.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
-import org.apache.carbondata.processing.store.CarbonFactHandler;
-import org.apache.carbondata.processing.store.CarbonFactHandlerFactory;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * It reads data from sorted files which are generated in previous sort step.
- * And it writes data to carbondata file. It also generates mdk key while writing to carbondata file
- */
-public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonRowDataWriterProcessorStepImpl.class.getName());
-
-  private int dimensionWithComplexCount;
-
-  private int noDictWithComplextCount;
-
-  private boolean[] isNoDictionaryDimensionColumn;
-
-  private DataType[] measureDataType;
-
-  private int dimensionCount;
-
-  private int measureCount;
-
-  private long[] readCounter;
-
-  private long[] writeCounter;
-
-  private int outputLength;
-
-  private CarbonTableIdentifier tableIdentifier;
-
-  private String tableName;
-
-  public CarbonRowDataWriterProcessorStepImpl(CarbonDataLoadConfiguration configuration,
-      AbstractDataLoadProcessorStep child) {
-    super(configuration, child);
-  }
-
-  @Override public DataField[] getOutput() {
-    return child.getOutput();
-  }
-
-  @Override public void initialize() throws IOException {
-    super.initialize();
-    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;
-  }
-
-  @Override public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
-    final Iterator<CarbonRowBatch>[] iterators = child.execute();
-    tableIdentifier = configuration.getTableIdentifier().getCarbonTableIdentifier();
-    tableName = tableIdentifier.getTableName();
-    try {
-      readCounter = new long[iterators.length];
-      writeCounter = new long[iterators.length];
-      dimensionWithComplexCount = configuration.getDimensionCount();
-      noDictWithComplextCount =
-          configuration.getNoDictionaryCount() + configuration.getComplexColumnCount();
-      dimensionCount = configuration.getDimensionCount() - noDictWithComplextCount;
-      isNoDictionaryDimensionColumn =
-          CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
-      measureDataType = configuration.getMeasureDataType();
-      CarbonFactDataHandlerModel dataHandlerModel = CarbonFactDataHandlerModel
-          .createCarbonFactDataHandlerModel(configuration,
-              getStoreLocation(tableIdentifier, String.valueOf(0)), 0, 0);
-      measureCount = dataHandlerModel.getMeasureCount();
-      outputLength = measureCount + (this.noDictWithComplextCount > 0 ? 1 : 0) + 1;
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
-              System.currentTimeMillis());
-
-      if (iterators.length == 1) {
-        doExecute(iterators[0], 0, 0);
-      } else {
-        ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
-        Future[] futures = new Future[iterators.length];
-        for (int i = 0; i < iterators.length; i++) {
-          futures[i] = executorService.submit(new DataWriterRunnable(iterators[i], i));
-        }
-        for (Future future : futures) {
-          future.get();
-        }
-      }
-    } catch (CarbonDataWriterException e) {
-      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
-      throw new CarbonDataLoadingException(
-          "Error while initializing data handler : " + e.getMessage());
-    } catch (Exception e) {
-      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
-      if (e instanceof BadRecordFoundException) {
-        throw new BadRecordFoundException(e.getMessage(), e);
-      }
-      throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage(), e);
-    }
-    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);
-    CarbonFactHandler dataHandler = null;
-    boolean rowsNotExist = true;
-    while (iterator.hasNext()) {
-      if (rowsNotExist) {
-        rowsNotExist = false;
-        dataHandler = CarbonFactHandlerFactory
-            .createCarbonFactHandler(model, CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
-        dataHandler.initialise();
-      }
-      processBatch(iterator.next(), dataHandler, iteratorIndex);
-    }
-    if (!rowsNotExist) {
-      finish(dataHandler, iteratorIndex);
-    }
-  }
-
-  @Override protected String getStepName() {
-    return "Data Writer";
-  }
-
-  private void finish(CarbonFactHandler dataHandler, int iteratorIndex) {
-    try {
-      dataHandler.finish();
-    } catch (Exception e) {
-      LOGGER.error(e, "Failed for table: " + tableName + " in  finishing data handler");
-    }
-    LOGGER.info("Record Processed For table: " + tableName);
-    String logMessage =
-        "Finished Carbon DataWriterProcessorStepImpl: Read: " + readCounter[iteratorIndex]
-            + ": Write: " + readCounter[iteratorIndex];
-    LOGGER.info(logMessage);
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
-    processingComplete(dataHandler);
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
-            System.currentTimeMillis());
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
-  }
-
-  private void processingComplete(CarbonFactHandler dataHandler) throws CarbonDataLoadingException {
-    if (null != dataHandler) {
-      try {
-        dataHandler.closeHandler();
-      } catch (CarbonDataWriterException e) {
-        LOGGER.error(e, e.getMessage());
-        throw new CarbonDataLoadingException(e.getMessage());
-      } catch (Exception e) {
-        LOGGER.error(e, e.getMessage());
-        throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage());
-      }
-    }
-  }
-
-  /**
-   * convert input CarbonRow to output CarbonRow
-   * e.g. There is a table as following,
-   * the number of dictionary dimensions is a,
-   * the number of no-dictionary dimensions is b,
-   * the number of complex dimensions is c,
-   * the number of measures is d.
-   * input CarbonRow format:  the length of Object[] data is a+b+c+d, the number of all columns.
-   * ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
-   * | Part                     | Object item                    | describe                 |
-   * ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
-   * | Object[0 ~ a+b-1]        | Integer, byte[], Integer, ...  | dict + no dict dimensions|
-   * ----------------------------------------------------------------------------------------
-   * | Object[a+b ~ a+b+c-1]    | byte[], byte[], ...            | complex dimensions       |
-   * ----------------------------------------------------------------------------------------
-   * | Object[a+b+c ~ a+b+c+d-1]| int, byte[], ...               | measures                 |
-   * ----------------------------------------------------------------------------------------
-   * output CarbonRow format: the length of object[] data is d + (b+c>0?1:0) + 1.
-   * ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
-   * | Part                     | Object item                    | describe                 |
-   * ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
-   * | Object[d+1]              | byte[]                         | mdkey                    |
-   * ----------------------------------------------------------------------------------------
-   * | Object[d]                | byte[b+c][]                    | no dict + complex dim    |
-   * ----------------------------------------------------------------------------------------
-   * | Object[0 ~ d-1]          | int, byte[], ...               | measures                 |
-   * ----------------------------------------------------------------------------------------
-   *
-   * @param row
-   * @return
-   */
-  private CarbonRow convertRow(CarbonRow row) throws KeyGenException {
-    int dictIndex = 0;
-    int nonDicIndex = 0;
-    int[] dim = new int[this.dimensionCount];
-    byte[][] nonDicArray = new byte[this.noDictWithComplextCount][];
-    // read dimension values
-    int dimCount = 0;
-    for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
-      if (isNoDictionaryDimensionColumn[dimCount]) {
-        nonDicArray[nonDicIndex++] = (byte[]) row.getObject(dimCount);
-      } else {
-        dim[dictIndex++] = (int) row.getObject(dimCount);
-      }
-    }
-
-    for (; dimCount < this.dimensionWithComplexCount; dimCount++) {
-      nonDicArray[nonDicIndex++] = (byte[]) row.getObject(dimCount);
-    }
-
-    Object[] measures = new Object[measureCount];
-    for (int i = 0; i < this.measureCount; i++) {
-      measures[i] = row.getObject(i + this.dimensionWithComplexCount);
-    }
-
-    return WriteStepRowUtil.fromColumnCategory(dim, nonDicArray, measures);
-  }
-
-  private void processBatch(CarbonRowBatch batch, CarbonFactHandler dataHandler, int iteratorIndex)
-      throws CarbonDataLoadingException {
-    try {
-      while (batch.hasNext()) {
-        CarbonRow row = batch.next();
-        CarbonRow converted = convertRow(row);
-        dataHandler.addDataToStore(converted);
-        readCounter[iteratorIndex]++;
-      }
-      writeCounter[iteratorIndex] += batch.getSize();
-    } catch (Exception e) {
-      throw new CarbonDataLoadingException("unable to generate the mdkey", e);
-    }
-    rowCounter.getAndAdd(batch.getSize());
-  }
-
-  @Override protected CarbonRow processRow(CarbonRow row) {
-    return null;
-  }
-
-  class DataWriterRunnable implements Runnable {
-
-    private Iterator<CarbonRowBatch> iterator;
-    private int iteratorIndex = 0;
-
-    DataWriterRunnable(Iterator<CarbonRowBatch> iterator, int iteratorIndex) {
-      this.iterator = iterator;
-      this.iteratorIndex = iteratorIndex;
-    }
-
-    @Override public void run() {
-      doExecute(this.iterator, 0, iteratorIndex);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java
deleted file mode 100644
index 728aa18..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorStepImpl.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.steps;
-
-import java.io.File;
-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.common.constants.LoggerAction;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
-import org.apache.carbondata.processing.newflow.converter.RowConverter;
-import org.apache.carbondata.processing.newflow.converter.impl.RowConverterImpl;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * 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 DataConverterProcessorStepImpl extends AbstractDataLoadProcessorStep {
-
-  private List<RowConverter> converters;
-  private BadRecordsLogger badRecordLogger;
-
-  public DataConverterProcessorStepImpl(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 = createBadRecordLogger(configuration);
-    RowConverter converter =
-        new RowConverterImpl(child.getOutput(), configuration, badRecordLogger);
-    configuration.setCardinalityFinder(converter);
-    converters.add(converter);
-    converter.initialize();
-  }
-
-  /**
-   * 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>() {
-      private boolean first = true;
-      private RowConverter localConverter;
-      @Override public boolean hasNext() {
-        if (first) {
-          first = false;
-          localConverter = converters.get(0).createCopyForNewThread();
-          synchronized (converters) {
-            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) {
-    CarbonRowBatch newBatch = new CarbonRowBatch(rowBatch.getSize());
-    while (rowBatch.hasNext()) {
-      newBatch.addRow(localConverter.convert(rowBatch.next()));
-    }
-    rowCounter.getAndAdd(newBatch.getSize());
-    return newBatch;
-  }
-
-  @Override
-  protected CarbonRow processRow(CarbonRow row) {
-    throw new UnsupportedOperationException();
-  }
-
-  public static BadRecordsLogger createBadRecordLogger(CarbonDataLoadConfiguration configuration) {
-    boolean badRecordsLogRedirect = false;
-    boolean badRecordConvertNullDisable = false;
-    boolean isDataLoadFail = false;
-    boolean badRecordsLoggerEnable = Boolean.parseBoolean(
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ENABLE)
-            .toString());
-    Object bad_records_action =
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ACTION)
-            .toString();
-    if (null != bad_records_action) {
-      LoggerAction loggerAction = null;
-      try {
-        loggerAction = LoggerAction.valueOf(bad_records_action.toString().toUpperCase());
-      } catch (IllegalArgumentException e) {
-        loggerAction = LoggerAction.FORCE;
-      }
-      switch (loggerAction) {
-        case FORCE:
-          badRecordConvertNullDisable = false;
-          break;
-        case REDIRECT:
-          badRecordsLogRedirect = true;
-          badRecordConvertNullDisable = true;
-          break;
-        case IGNORE:
-          badRecordsLogRedirect = false;
-          badRecordConvertNullDisable = true;
-          break;
-        case FAIL:
-          isDataLoadFail = true;
-          break;
-      }
-    }
-    CarbonTableIdentifier identifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    return new BadRecordsLogger(identifier.getBadRecordLoggerKey(),
-        identifier.getTableName() + '_' + System.currentTimeMillis(),
-        getBadLogStoreLocation(configuration,
-            identifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + identifier
-                .getTableName() + CarbonCommonConstants.FILE_SEPARATOR + configuration
-                .getSegmentId() + CarbonCommonConstants.FILE_SEPARATOR + configuration.getTaskNo()),
-        badRecordsLogRedirect, badRecordsLoggerEnable, badRecordConvertNullDisable, isDataLoadFail);
-  }
-
-  public static String getBadLogStoreLocation(CarbonDataLoadConfiguration configuration,
-      String storeLocation) {
-    String badLogStoreLocation = (String) configuration
-        .getDataLoadProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH);
-    if (null == badLogStoreLocation) {
-      badLogStoreLocation =
-          CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC);
-    }
-    badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation;
-
-    return badLogStoreLocation;
-  }
-
-  @Override
-  public void close() {
-    if (!closed) {
-      if (null != badRecordLogger) {
-        badRecordLogger.closeStreams();
-        renameBadRecord(badRecordLogger, configuration);
-      }
-      super.close();
-      if (converters != null) {
-        for (RowConverter converter : converters) {
-          if (null != converter) {
-            converter.finish();
-          }
-        }
-      }
-    }
-  }
-
-  public static void close(BadRecordsLogger badRecordLogger, CarbonDataLoadConfiguration
-      configuration, RowConverter converter) {
-    if (badRecordLogger != null) {
-      badRecordLogger.closeStreams();
-      renameBadRecord(badRecordLogger, configuration);
-    }
-    if (converter != null) {
-      converter.finish();
-    }
-  }
-
-  private static void renameBadRecord(BadRecordsLogger badRecordLogger,
-      CarbonDataLoadConfiguration configuration) {
-    // rename operation should be performed only in case either bad reccords loggers is enabled
-    // or bad records redirect is enabled
-    if (badRecordLogger.isBadRecordLoggerEnable() || badRecordLogger.isBadRecordsLogRedirect()) {
-      // rename the bad record in progress to normal
-      CarbonTableIdentifier identifier =
-          configuration.getTableIdentifier().getCarbonTableIdentifier();
-      CarbonDataProcessorUtil.renameBadRecordsFromInProgressToNormal(configuration,
-          identifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + identifier
-              .getTableName() + CarbonCommonConstants.FILE_SEPARATOR + configuration.getSegmentId()
-              + CarbonCommonConstants.FILE_SEPARATOR + configuration.getTaskNo());
-    }
-  }
-
-  @Override protected String getStepName() {
-    return "Data Converter";
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java
deleted file mode 100644
index fe36f82..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataConverterProcessorWithBucketingStepImpl.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.steps;
-
-import java.io.File;
-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.common.constants.LoggerAction;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.BucketingInfo;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
-import org.apache.carbondata.processing.newflow.converter.RowConverter;
-import org.apache.carbondata.processing.newflow.converter.impl.RowConverterImpl;
-import org.apache.carbondata.processing.newflow.partition.Partitioner;
-import org.apache.carbondata.processing.newflow.partition.impl.HashPartitionerImpl;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * 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 = createBadRecordLogger();
-    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) {
-    CarbonRowBatch newBatch = new CarbonRowBatch(rowBatch.getSize());
-    while (rowBatch.hasNext()) {
-      CarbonRow next = rowBatch.next();
-      short bucketNumber = (short) partitioner.getPartition(next.getData());
-      CarbonRow convertRow = localConverter.convert(next);
-      convertRow.bucketNumber = bucketNumber;
-      newBatch.addRow(convertRow);
-    }
-    rowCounter.getAndAdd(newBatch.getSize());
-    return newBatch;
-  }
-
-  @Override
-  protected CarbonRow processRow(CarbonRow row) {
-    throw new UnsupportedOperationException();
-  }
-
-  private BadRecordsLogger createBadRecordLogger() {
-    boolean badRecordsLogRedirect = false;
-    boolean badRecordConvertNullDisable = false;
-    boolean isDataLoadFail = false;
-    boolean badRecordsLoggerEnable = Boolean.parseBoolean(
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ENABLE)
-            .toString());
-    Object bad_records_action =
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ACTION)
-            .toString();
-    if (null != bad_records_action) {
-      LoggerAction loggerAction = null;
-      try {
-        loggerAction = LoggerAction.valueOf(bad_records_action.toString().toUpperCase());
-      } catch (IllegalArgumentException e) {
-        loggerAction = LoggerAction.FORCE;
-      }
-      switch (loggerAction) {
-        case FORCE:
-          badRecordConvertNullDisable = false;
-          break;
-        case REDIRECT:
-          badRecordsLogRedirect = true;
-          badRecordConvertNullDisable = true;
-          break;
-        case IGNORE:
-          badRecordsLogRedirect = false;
-          badRecordConvertNullDisable = true;
-          break;
-        case FAIL:
-          isDataLoadFail = true;
-          break;
-      }
-    }
-    CarbonTableIdentifier identifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    return new BadRecordsLogger(identifier.getBadRecordLoggerKey(),
-        identifier.getTableName() + '_' + System.currentTimeMillis(), getBadLogStoreLocation(
-        identifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + identifier
-            .getTableName() + CarbonCommonConstants.FILE_SEPARATOR + configuration.getSegmentId()
-            + CarbonCommonConstants.FILE_SEPARATOR + configuration.getTaskNo()),
-        badRecordsLogRedirect, badRecordsLoggerEnable, badRecordConvertNullDisable, isDataLoadFail);
-  }
-
-  private String getBadLogStoreLocation(String storeLocation) {
-    String badLogStoreLocation = (String) configuration
-        .getDataLoadProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH);
-    if (null == badLogStoreLocation) {
-      badLogStoreLocation =
-          CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC);
-    }
-    badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation;
-
-    return badLogStoreLocation;
-  }
-
-  @Override
-  public void close() {
-    if (!closed) {
-      super.close();
-      if (null != badRecordLogger) {
-        badRecordLogger.closeStreams();
-        renameBadRecord(configuration);
-      }
-      if (converters != null) {
-        for (RowConverter converter : converters) {
-          converter.finish();
-        }
-      }
-    }
-  }
-  private static void renameBadRecord(CarbonDataLoadConfiguration configuration) {
-    // rename the bad record in progress to normal
-    CarbonTableIdentifier identifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    CarbonDataProcessorUtil.renameBadRecordsFromInProgressToNormal(configuration,
-        identifier.getDatabaseName() + File.separator + identifier.getTableName()
-            + File.separator + configuration.getSegmentId() + File.separator + configuration
-            .getTaskNo());
-  }
-  @Override protected String getStepName() {
-    return "Data Converter with Bucketing";
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterBatchProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterBatchProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterBatchProcessorStepImpl.java
deleted file mode 100644
index b13cd26..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/steps/DataWriterBatchProcessorStepImpl.java
+++ /dev/null
@@ -1,155 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.steps;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-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.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
-import org.apache.carbondata.processing.store.CarbonFactHandler;
-import org.apache.carbondata.processing.store.CarbonFactHandlerFactory;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * It reads data from batch of sorted files(it could be in-memory/disk based files)
- * which are generated in previous sort step. And it writes data to carbondata file.
- * It also generates mdk key while writing to carbondata file
- */
-public class DataWriterBatchProcessorStepImpl extends AbstractDataLoadProcessorStep {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DataWriterBatchProcessorStepImpl.class.getName());
-
-  public DataWriterBatchProcessorStepImpl(CarbonDataLoadConfiguration configuration,
-      AbstractDataLoadProcessorStep child) {
-    super(configuration, child);
-  }
-
-  @Override public DataField[] getOutput() {
-    return child.getOutput();
-  }
-
-  @Override public void initialize() throws IOException {
-    super.initialize();
-    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;
-  }
-
-  @Override public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
-    Iterator<CarbonRowBatch>[] iterators = child.execute();
-    CarbonTableIdentifier tableIdentifier =
-        configuration.getTableIdentifier().getCarbonTableIdentifier();
-    String tableName = tableIdentifier.getTableName();
-    try {
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
-              System.currentTimeMillis());
-      int i = 0;
-      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++);
-            CarbonFactHandler dataHandler = CarbonFactHandlerFactory
-                .createCarbonFactHandler(model, CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
-            dataHandler.initialise();
-            processBatch(next, dataHandler);
-            finish(tableName, dataHandler);
-          }
-        }
-        i++;
-      }
-    } catch (Exception e) {
-      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterBatchProcessorStepImpl");
-      if (e.getCause() instanceof BadRecordFoundException) {
-        throw new BadRecordFoundException(e.getCause().getMessage());
-      }
-      throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage());
-    }
-    return null;
-  }
-
-  @Override protected String getStepName() {
-    return "Data Batch Writer";
-  }
-
-  private void finish(String tableName, CarbonFactHandler dataHandler) {
-    try {
-      dataHandler.finish();
-    } catch (Exception e) {
-      LOGGER.error(e, "Failed for table: " + tableName + " in  finishing data handler");
-    }
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
-    processingComplete(dataHandler);
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
-            System.currentTimeMillis());
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
-  }
-
-  private void processingComplete(CarbonFactHandler dataHandler) {
-    if (null != dataHandler) {
-      try {
-        dataHandler.closeHandler();
-      } catch (Exception e) {
-        LOGGER.error(e);
-        throw new CarbonDataLoadingException(
-            "There is an unexpected error while closing data handler", e);
-      }
-    }
-  }
-
-  private void processBatch(CarbonRowBatch batch, CarbonFactHandler dataHandler) throws Exception {
-    int batchSize = 0;
-    while (batch.hasNext()) {
-      CarbonRow row = batch.next();
-      dataHandler.addDataToStore(row);
-      batchSize++;
-    }
-    batch.close();
-    rowCounter.getAndAdd(batchSize);
-  }
-
-  @Override protected CarbonRow processRow(CarbonRow row) {
-    return null;
-  }
-
-}


[12/50] [abbrv] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by ja...@apache.org.
[CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

This closes #1379


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

Branch: refs/heads/streaming_ingest
Commit: 6627cac0ccdfa8cc94cda188f211112e82ec3294
Parents: 2ec69f6
Author: Jacky Li <ja...@qq.com>
Authored: Sun Sep 24 23:43:20 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Wed Sep 27 14:32:20 2017 +0800

----------------------------------------------------------------------
 .../spark/sql/CarbonCatalystOperators.scala     |   16 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |    4 +-
 .../sql/CarbonDatasourceHadoopRelation.scala    |    4 +-
 .../org/apache/spark/sql/CarbonSource.scala     |    2 +-
 .../spark/sql/SparkUnknownExpression.scala      |    1 -
 .../execution/CarbonLateDecodeStrategy.scala    |  553 --------
 .../execution/command/AlterTableCommands.scala  |  460 ------
 .../command/CarbonCreateTableCommand.scala      |  107 ++
 .../CarbonDescribeFormattedCommand.scala        |  133 ++
 .../command/CarbonDropTableCommand.scala        |   94 ++
 .../sql/execution/command/DDLStrategy.scala     |  143 --
 .../sql/execution/command/IUDCommands.scala     |  857 ------------
 .../execution/command/carbonTableSchema.scala   | 1315 ------------------
 .../AlterTableCompactionCommand.scala           |   90 ++
 .../management/CarbonShowLoadsCommand.scala     |   50 +
 .../command/management/CleanFilesCommand.scala  |   58 +
 .../management/DeleteLoadByIdCommand.scala      |   48 +
 .../DeleteLoadByLoadDateCommand.scala           |   50 +
 .../management/LoadTableByInsertCommand.scala   |   53 +
 .../command/management/LoadTableCommand.scala   |  520 +++++++
 .../command/mutation/DeleteExecution.scala      |  322 +++++
 .../command/mutation/HorizontalCompaction.scala |  246 ++++
 .../HorizontalCompactionException.scala         |   24 +
 .../mutation/ProjectForDeleteCommand.scala      |  105 ++
 .../mutation/ProjectForUpdateCommand.scala      |  228 +++
 .../spark/sql/execution/command/package.scala   |   53 +
 .../AlterTableDropCarbonPartitionCommand.scala  |  176 +++
 .../AlterTableSplitCarbonPartitionCommand.scala |  187 +++
 .../partition/ShowCarbonPartitionsCommand.scala |   60 +
 .../schema/AlterTableAddColumnCommand.scala     |  115 ++
 .../AlterTableDataTypeChangeCommand.scala       |  116 ++
 .../schema/AlterTableDropColumnCommand.scala    |  148 ++
 .../schema/AlterTableRenameTableCommand.scala   |  174 +++
 .../strategy/CarbonLateDecodeStrategy.scala     |  554 ++++++++
 .../sql/execution/strategy/DDLStrategy.scala    |  162 +++
 .../spark/sql/hive/CarbonAnalysisRules.scala    |   10 +-
 .../spark/sql/hive/CarbonSessionState.scala     |    7 +-
 .../execution/command/CarbonHiveCommands.scala  |    5 +-
 .../sql/optimizer/CarbonLateDecodeRule.scala    |    3 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   41 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala |    4 +-
 .../org/apache/spark/util/Compaction.scala      |    5 +-
 .../org/apache/spark/util/TableLoader.scala     |    4 +-
 .../vectorreader/VectorReaderTestCase.scala     |    2 +-
 44 files changed, 3928 insertions(+), 3381 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index 0c3414a..6815629 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -65,7 +65,7 @@ case class IncludeProfile(attributes: Seq[Attribute]) extends CarbonProfile(attr
 
 case class ExcludeProfile(attributes: Seq[Attribute]) extends CarbonProfile(attributes)
 
-object getDB {
+object GetDB {
 
   def getDatabaseName(dbName: Option[String], sparkSession: SparkSession): String = {
     dbName.getOrElse(
@@ -76,7 +76,10 @@ object getDB {
 /**
  * Shows Loads in a table
  */
-case class ShowLoadsCommand(databaseNameOp: Option[String], table: String, limit: Option[String])
+case class ShowLoadsCommand(
+    databaseNameOp: Option[String],
+    table: String,
+    limit: Option[String])
   extends Command {
 
   override def output: Seq[Attribute] = {
@@ -111,15 +114,6 @@ case class DeleteRecords(
 }
 
 /**
- * Describe formatted for hive table
- */
-case class DescribeFormattedCommand(sql: String, tblIdentifier: TableIdentifier) extends Command {
-
-  override def output: Seq[AttributeReference] =
-    Seq(AttributeReference("result", StringType, nullable = false)())
-}
-
-/**
  * A logical plan representing insertion into Hive table
  * This plan ignores nullability of ArrayType, MapType, StructType unlike InsertIntoTable
  * because Hive Table doesn't have nullability for ARRAY, MAP,STRUCT types.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
index e9f2758..a12d86b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDataFrameWriter.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql
 
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.compress.GzipCodec
-import org.apache.spark.sql.execution.command.LoadTable
+import org.apache.spark.sql.execution.command.management.LoadTableCommand
 import org.apache.spark.sql.types._
 
 import org.apache.carbondata.common.logging.LogServiceFactory
@@ -130,7 +130,7 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
    */
   private def loadDataFrame(options: CarbonOption): Unit = {
     val header = dataFrame.columns.mkString(",")
-    LoadTable(
+    LoadTableCommand(
       Some(options.dbName),
       options.tableName,
       null,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
index 4f4515d..d5adc2f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.CarbonInputMetrics
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.execution.command.LoadTableByInsert
+import org.apache.spark.sql.execution.command.management.LoadTableByInsertCommand
 import org.apache.spark.sql.hive.CarbonRelation
 import org.apache.spark.sql.optimizer.CarbonFilters
 import org.apache.spark.sql.sources.{BaseRelation, Filter, InsertableRelation}
@@ -95,7 +95,7 @@ case class CarbonDatasourceHadoopRelation(
           CarbonCommonConstants.DEFAULT_MAX_NUMBER_OF_COLUMNS)
     }
     if (data.logicalPlan.output.size >= carbonRelation.output.size) {
-      LoadTableByInsert(this, data.logicalPlan, overwrite).run(sparkSession)
+      LoadTableByInsertCommand(this, data.logicalPlan, overwrite).run(sparkSession)
     } else {
       sys.error("Cannot insert into target table because column number are different")
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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 1b021b0..f4f8b75 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
@@ -24,8 +24,8 @@ import org.apache.commons.lang.StringUtils
 import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
-import org.apache.spark.sql.execution.CarbonLateDecodeStrategy
 import org.apache.spark.sql.execution.command.{TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.strategy.CarbonLateDecodeStrategy
 import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
 import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
index 50d7dba..d536746 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
@@ -21,7 +21,6 @@ import java.util.{ArrayList, List}
 
 import scala.collection.JavaConverters._
 
-import org.apache.spark.sql.CarbonBoundReference
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression, GenericInternalRow}
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
deleted file mode 100644
index 4d8e7ac..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/CarbonLateDecodeStrategy.scala
+++ /dev/null
@@ -1,553 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.expressions.{Attribute, _}
-import org.apache.spark.sql.catalyst.planning.PhysicalOperation
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.optimizer.CarbonDecoderRelation
-import org.apache.spark.sql.sources.{BaseRelation, Filter}
-import org.apache.spark.sql.types.{AtomicType, IntegerType, StringType}
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.schema.BucketingInfo
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.spark.CarbonAliasDecoderRelation
-import org.apache.carbondata.spark.rdd.CarbonScanRDD
-import org.apache.carbondata.spark.util.CarbonScalaUtil
-
-/**
- * Carbon specific optimization for late decode (convert dictionary key to value as late as
- * possible), which can improve the aggregation performance and reduce memory usage
- */
-private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
-  val PUSHED_FILTERS = "PushedFilters"
-
-  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
-    plan match {
-      case PhysicalOperation(projects, filters, l: LogicalRelation)
-        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
-        val relation = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-        pruneFilterProject(
-          l,
-          projects,
-          filters,
-          (a, f, needDecoder) => toCatalystRDD(l, a, relation.buildScan(
-            a.map(_.name).toArray, f), needDecoder)) :: Nil
-      case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
-        if ((profile.isInstanceOf[IncludeProfile] && profile.isEmpty) ||
-            !CarbonDictionaryDecoder.
-              isRequiredToDecode(CarbonDictionaryDecoder.
-                getDictionaryColumnMapping(child.output, relations, profile, aliasMap))) {
-          planLater(child) :: Nil
-        } else {
-          CarbonDictionaryDecoder(relations,
-            profile,
-            aliasMap,
-            planLater(child),
-            SparkSession.getActiveSession.get
-          ) :: Nil
-        }
-      case _ => Nil
-    }
-  }
-
-
-  def getDecoderRDD(
-      logicalRelation: LogicalRelation,
-      projectExprsNeedToDecode: ArrayBuffer[AttributeReference],
-      rdd: RDD[InternalRow],
-      output: Seq[Attribute]): RDD[InternalRow] = {
-    val table = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-    val relation = CarbonDecoderRelation(logicalRelation.attributeMap,
-      logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation])
-    val attrs = projectExprsNeedToDecode.map { attr =>
-      val newAttr = AttributeReference(attr.name,
-        attr.dataType,
-        attr.nullable,
-        attr.metadata)(attr.exprId, Option(table.carbonRelation.tableName))
-      relation.addAttribute(newAttr)
-      newAttr
-    }
-
-    new CarbonDecoderRDD(
-      Seq(relation),
-      IncludeProfile(attrs),
-      CarbonAliasDecoderRelation(),
-      rdd,
-      output,
-      CarbonEnv.getInstance(SparkSession.getActiveSession.get).storePath,
-      table.carbonTable.getTableInfo.serialize())
-  }
-
-  private[this] def toCatalystRDD(
-      relation: LogicalRelation,
-      output: Seq[Attribute],
-      rdd: RDD[InternalRow],
-      needDecode: ArrayBuffer[AttributeReference]):
-  RDD[InternalRow] = {
-    if (needDecode.nonEmpty) {
-      rdd.asInstanceOf[CarbonScanRDD].setVectorReaderSupport(false)
-      getDecoderRDD(relation, needDecode, rdd, output)
-    } else {
-      rdd.asInstanceOf[CarbonScanRDD]
-        .setVectorReaderSupport(supportBatchedDataSource(relation.relation.sqlContext, output))
-      rdd
-    }
-  }
-
-  protected def pruneFilterProject(
-      relation: LogicalRelation,
-      projects: Seq[NamedExpression],
-      filterPredicates: Seq[Expression],
-      scanBuilder: (Seq[Attribute], Array[Filter],
-        ArrayBuffer[AttributeReference]) => RDD[InternalRow]) = {
-    pruneFilterProjectRaw(
-      relation,
-      projects,
-      filterPredicates,
-      (requestedColumns, _, pushedFilters, a) => {
-        scanBuilder(requestedColumns, pushedFilters.toArray, a)
-      })
-  }
-
-  protected def pruneFilterProjectRaw(
-      relation: LogicalRelation,
-      rawProjects: Seq[NamedExpression],
-      filterPredicates: Seq[Expression],
-      scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
-        ArrayBuffer[AttributeReference]) => RDD[InternalRow]) = {
-    val projects = rawProjects.map {p =>
-      p.transform {
-        case CustomDeterministicExpression(exp) => exp
-      }
-    }.asInstanceOf[Seq[NamedExpression]]
-
-    val projectSet = AttributeSet(projects.flatMap(_.references))
-    val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
-
-    val candidatePredicates = filterPredicates.map {
-      _ transform {
-        case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes.
-      }
-    }
-
-    val (unhandledPredicates, pushedFilters) =
-      selectFilters(relation.relation, candidatePredicates)
-
-    // A set of column attributes that are only referenced by pushed down filters.  We can eliminate
-    // them from requested columns.
-    val handledSet = {
-      val handledPredicates = filterPredicates.filterNot(unhandledPredicates.contains)
-      val unhandledSet = AttributeSet(unhandledPredicates.flatMap(_.references))
-      AttributeSet(handledPredicates.flatMap(_.references)) --
-      (projectSet ++ unhandledSet).map(relation.attributeMap)
-    }
-
-    // Combines all Catalyst filter `Expression`s that are either not convertible to data source
-    // `Filter`s or cannot be handled by `relation`.
-    val filterCondition = unhandledPredicates.reduceLeftOption(expressions.And)
-    val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-    val map = table.carbonRelation.metaData.dictionaryMap
-
-    val metadata: Map[String, String] = {
-      val pairs = ArrayBuffer.empty[(String, String)]
-
-      if (pushedFilters.nonEmpty) {
-        pairs += (PUSHED_FILTERS -> pushedFilters.mkString("[", ", ", "]"))
-      }
-      pairs.toMap
-    }
-
-
-    val needDecoder = ArrayBuffer[AttributeReference]()
-    filterCondition match {
-      case Some(exp: Expression) =>
-        exp.references.collect {
-          case attr: AttributeReference =>
-            val dict = map.get(attr.name)
-            if (dict.isDefined && dict.get) {
-              needDecoder += attr
-            }
-        }
-      case None =>
-    }
-
-    projects.map {
-      case attr: AttributeReference =>
-      case Alias(attr: AttributeReference, _) =>
-      case others =>
-        others.references.map { f =>
-          val dictionary = map.get(f.name)
-          if (dictionary.isDefined && dictionary.get) {
-            needDecoder += f.asInstanceOf[AttributeReference]
-          }
-        }
-    }
-
-    if (projects.map(_.toAttribute) == projects &&
-        projectSet.size == projects.size &&
-        filterSet.subsetOf(projectSet)) {
-      // When it is possible to just use column pruning to get the right projection and
-      // when the columns of this projection are enough to evaluate all filter conditions,
-      // just do a scan followed by a filter, with no extra project.
-      val requestedColumns = projects
-        // Safe due to if above.
-        .asInstanceOf[Seq[Attribute]]
-        // Match original case of attributes.
-        .map(relation.attributeMap)
-        // Don't request columns that are only referenced by pushed filters.
-        .filterNot(handledSet.contains)
-      val updateRequestedColumns = updateRequestedColumnsFunc(requestedColumns, table, needDecoder)
-
-      val updateProject = projects.map { expr =>
-        var attr = expr.toAttribute.asInstanceOf[AttributeReference]
-        if (!needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
-          val dict = map.get(attr.name)
-          if (dict.isDefined && dict.get) {
-            attr = AttributeReference(attr.name, IntegerType, attr.nullable, attr.metadata)(attr
-              .exprId, attr.qualifier)
-          }
-        }
-        attr
-      }
-      val scan = getDataSourceScan(relation,
-        updateProject,
-        scanBuilder,
-        candidatePredicates,
-        pushedFilters,
-        metadata,
-        needDecoder,
-        updateRequestedColumns.asInstanceOf[Seq[Attribute]])
-      filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan)
-    } else {
-
-      var newProjectList: Seq[Attribute] = Seq.empty
-      val updatedProjects = projects.map {
-          case a@Alias(s: ScalaUDF, name)
-            if name.equalsIgnoreCase(CarbonCommonConstants.POSITION_ID) ||
-                name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID) =>
-            val reference = AttributeReference(name, StringType, true)().withExprId(a.exprId)
-            newProjectList :+= reference
-            reference
-          case a@Alias(s: ScalaUDF, name)
-            if name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID) =>
-            val reference =
-              AttributeReference(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID,
-                StringType, true)().withExprId(a.exprId)
-            newProjectList :+= reference
-            a.transform {
-              case s: ScalaUDF =>
-                ScalaUDF(s.function, s.dataType, Seq(reference), s.inputTypes)
-            }
-          case other => other
-      }
-      // Don't request columns that are only referenced by pushed filters.
-      val requestedColumns =
-        (projectSet ++ filterSet -- handledSet).map(relation.attributeMap).toSeq ++ newProjectList
-      val updateRequestedColumns = updateRequestedColumnsFunc(requestedColumns, table, needDecoder)
-      val scan = getDataSourceScan(relation,
-        updateRequestedColumns.asInstanceOf[Seq[Attribute]],
-        scanBuilder,
-        candidatePredicates,
-        pushedFilters,
-        metadata,
-        needDecoder,
-        updateRequestedColumns.asInstanceOf[Seq[Attribute]])
-      execution.ProjectExec(
-        updateRequestedColumnsFunc(updatedProjects, table,
-          needDecoder).asInstanceOf[Seq[NamedExpression]],
-        filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan))
-    }
-  }
-
-  def getDataSourceScan(relation: LogicalRelation,
-      output: Seq[Attribute],
-      scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
-        ArrayBuffer[AttributeReference]) => RDD[InternalRow],
-      candidatePredicates: Seq[Expression],
-      pushedFilters: Seq[Filter],
-      metadata: Map[String, String],
-      needDecoder: ArrayBuffer[AttributeReference],
-      updateRequestedColumns: Seq[Attribute]): DataSourceScanExec = {
-    val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
-    if (supportBatchedDataSource(relation.relation.sqlContext, updateRequestedColumns) &&
-        needDecoder.isEmpty) {
-      BatchedDataSourceScanExec(
-        output,
-        scanBuilder(updateRequestedColumns, candidatePredicates, pushedFilters, needDecoder),
-        relation.relation,
-        getPartitioning(table.carbonTable, updateRequestedColumns),
-        metadata,
-        relation.catalogTable.map(_.identifier))
-    } else {
-      RowDataSourceScanExec(output,
-        scanBuilder(updateRequestedColumns, candidatePredicates, pushedFilters, needDecoder),
-        relation.relation,
-        getPartitioning(table.carbonTable, updateRequestedColumns),
-        metadata,
-        relation.catalogTable.map(_.identifier))
-    }
-  }
-
-  def updateRequestedColumnsFunc(requestedColumns: Seq[Expression],
-      relation: CarbonDatasourceHadoopRelation,
-      needDecoder: ArrayBuffer[AttributeReference]): Seq[Expression] = {
-    val map = relation.carbonRelation.metaData.dictionaryMap
-    requestedColumns.map {
-      case attr: AttributeReference =>
-        if (needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
-          attr
-        } else {
-          val dict = map.get(attr.name)
-          if (dict.isDefined && dict.get) {
-            AttributeReference(attr.name,
-              IntegerType,
-              attr.nullable,
-              attr.metadata)(attr.exprId, attr.qualifier)
-          } else {
-            attr
-          }
-        }
-      case alias @ Alias(attr: AttributeReference, name) =>
-        if (needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
-          alias
-        } else {
-          val dict = map.get(attr.name)
-          if (dict.isDefined && dict.get) {
-            alias.transform {
-              case attrLocal: AttributeReference =>
-                AttributeReference(attr.name,
-                  IntegerType,
-                  attr.nullable,
-                  attr.metadata)(attr.exprId, attr.qualifier)
-            }
-          } else {
-            alias
-          }
-        }
-      case others => others
-    }
-  }
-
-  private def getPartitioning(carbonTable: CarbonTable,
-      output: Seq[Attribute]): Partitioning = {
-    val info: BucketingInfo = carbonTable.getBucketingInfo(carbonTable.getFactTableName)
-    if (info != null) {
-      val cols = info.getListOfColumns.asScala
-      val sortColumn = carbonTable.
-        getDimensionByTableName(carbonTable.getFactTableName).get(0).getColName
-      val numBuckets = info.getNumberOfBuckets
-      val bucketColumns = cols.flatMap { n =>
-        val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
-        attrRef match {
-          case Some(attr: AttributeReference) =>
-            Some(AttributeReference(attr.name,
-              CarbonScalaUtil.convertCarbonToSparkDataType(n.getDataType),
-              attr.nullable,
-              attr.metadata)(attr.exprId, attr.qualifier))
-          case _ => None
-        }
-      }
-      if (bucketColumns.size == cols.size) {
-        HashPartitioning(bucketColumns, numBuckets)
-      } else {
-        UnknownPartitioning(0)
-      }
-    } else {
-      UnknownPartitioning(0)
-    }
-  }
-
-  protected[sql] def selectFilters(
-      relation: BaseRelation,
-      predicates: Seq[Expression]): (Seq[Expression], Seq[Filter]) = {
-
-    // For conciseness, all Catalyst filter expressions of type `expressions.Expression` below are
-    // called `predicate`s, while all data source filters of type `sources.Filter` are simply called
-    // `filter`s.
-
-    val translated: Seq[(Expression, Filter)] =
-      for {
-        predicate <- predicates
-        filter <- translateFilter(predicate)
-      } yield predicate -> filter
-
-    // A map from original Catalyst expressions to corresponding translated data source filters.
-    val translatedMap: Map[Expression, Filter] = translated.toMap
-
-    // Catalyst predicate expressions that cannot be translated to data source filters.
-    val unrecognizedPredicates = predicates.filterNot(translatedMap.contains)
-
-    // Data source filters that cannot be handled by `relation`. The semantic of a unhandled filter
-    // at here is that a data source may not be able to apply this filter to every row
-    // of the underlying dataset.
-    val unhandledFilters = relation.unhandledFilters(translatedMap.values.toArray).toSet
-
-    val (unhandled, handled) = translated.partition {
-      case (predicate, filter) =>
-        unhandledFilters.contains(filter)
-    }
-
-    // Catalyst predicate expressions that can be translated to data source filters, but cannot be
-    // handled by `relation`.
-    val (unhandledPredicates, _) = unhandled.unzip
-
-    // Translated data source filters that can be handled by `relation`
-    val (_, handledFilters) = handled.unzip
-
-    // translated contains all filters that have been converted to the public Filter interface.
-    // We should always push them to the data source no matter whether the data source can apply
-    // a filter to every row or not.
-    val (_, translatedFilters) = translated.unzip
-
-    (unrecognizedPredicates ++ unhandledPredicates, translatedFilters)
-  }
-
-
-  /**
-   * Tries to translate a Catalyst [[Expression]] into data source [[Filter]].
-   *
-   * @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`.
-   */
-  protected[sql] def translateFilter(predicate: Expression, or: Boolean = false): Option[Filter] = {
-    predicate match {
-      case or@Or(left, right) =>
-
-        val leftFilter = translateFilter(left, true)
-        val rightFilter = translateFilter(right, true)
-        if (leftFilter.isDefined && rightFilter.isDefined) {
-          Some(sources.Or(leftFilter.get, rightFilter.get))
-        } else {
-          None
-        }
-
-      case And(left, right) =>
-        val leftFilter = translateFilter(left, or)
-        val rightFilter = translateFilter(right, or)
-        if (or) {
-          if (leftFilter.isDefined && rightFilter.isDefined) {
-            (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
-          } else {
-            None
-          }
-        } else {
-          (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
-        }
-      case EqualTo(a: Attribute, Literal(v, t)) =>
-        Some(sources.EqualTo(a.name, v))
-      case EqualTo(l@Literal(v, t), a: Attribute) =>
-        Some(sources.EqualTo(a.name, v))
-      case c@EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case Not(EqualTo(a: Attribute, Literal(v, t))) =>
-        Some(sources.Not(sources.EqualTo(a.name, v)))
-      case Not(EqualTo(Literal(v, t), a: Attribute)) =>
-        Some(sources.Not(sources.EqualTo(a.name, v)))
-      case c@Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case IsNotNull(a: Attribute) => Some(sources.IsNotNull(a.name))
-      case IsNull(a: Attribute) => Some(sources.IsNull(a.name))
-      case Not(In(a: Attribute, list)) if !list.exists(!_.isInstanceOf[Literal]) =>
-        val hSet = list.map(e => e.eval(EmptyRow))
-        Some(sources.Not(sources.In(a.name, hSet.toArray)))
-      case In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) =>
-        val hSet = list.map(e => e.eval(EmptyRow))
-        Some(sources.In(a.name, hSet.toArray))
-      case c@Not(In(Cast(a: Attribute, _), list))
-        if !list.exists(!_.isInstanceOf[Literal]) =>
-        Some(CastExpr(c))
-      case c@In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
-        Some(CastExpr(c))
-      case InSet(a: Attribute, set) =>
-        Some(sources.In(a.name, set.toArray))
-      case Not(InSet(a: Attribute, set)) =>
-        Some(sources.Not(sources.In(a.name, set.toArray)))
-      case GreaterThan(a: Attribute, Literal(v, t)) =>
-        Some(sources.GreaterThan(a.name, v))
-      case GreaterThan(Literal(v, t), a: Attribute) =>
-        Some(sources.LessThan(a.name, v))
-      case c@GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case LessThan(a: Attribute, Literal(v, t)) =>
-        Some(sources.LessThan(a.name, v))
-      case LessThan(Literal(v, t), a: Attribute) =>
-        Some(sources.GreaterThan(a.name, v))
-      case c@LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case GreaterThanOrEqual(a: Attribute, Literal(v, t)) =>
-        Some(sources.GreaterThanOrEqual(a.name, v))
-      case GreaterThanOrEqual(Literal(v, t), a: Attribute) =>
-        Some(sources.LessThanOrEqual(a.name, v))
-      case c@GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case LessThanOrEqual(a: Attribute, Literal(v, t)) =>
-        Some(sources.LessThanOrEqual(a.name, v))
-      case LessThanOrEqual(Literal(v, t), a: Attribute) =>
-        Some(sources.GreaterThanOrEqual(a.name, v))
-      case c@LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
-        CastExpressionOptimization.checkIfCastCanBeRemove(c)
-      case StartsWith(a: Attribute, Literal(v, t)) =>
-        Some(sources.StringStartsWith(a.name, v.toString))
-      case c@EndsWith(a: Attribute, Literal(v, t)) =>
-        Some(CarbonEndsWith(c))
-      case c@Contains(a: Attribute, Literal(v, t)) =>
-        Some(CarbonContainsWith(c))
-      case others => None
-    }
-  }
-
-  def supportBatchedDataSource(sqlContext: SQLContext, cols: Seq[Attribute]): Boolean = {
-    val vectorizedReader = {
-      if (sqlContext.sparkSession.conf.contains(CarbonCommonConstants.ENABLE_VECTOR_READER)) {
-        sqlContext.sparkSession.conf.get(CarbonCommonConstants.ENABLE_VECTOR_READER)
-      } else if (System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER) != null) {
-        System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER)
-      } else {
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER,
-          CarbonCommonConstants.ENABLE_VECTOR_READER_DEFAULT)
-      }
-    }
-    val supportCodegen =
-      sqlContext.conf.wholeStageEnabled && sqlContext.conf.wholeStageMaxNumFields >= cols.size
-    supportCodegen && vectorizedReader.toBoolean &&
-    cols.forall(_.dataType.isInstanceOf[AtomicType])
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
deleted file mode 100644
index 17e456d..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/AlterTableCommands.scala
+++ /dev/null
@@ -1,460 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.execution.command
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ListBuffer
-import scala.language.implicitConversions
-
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
-import org.apache.spark.util.AlterTableUtil
-
-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.locks.{ICarbonLock, LockUsage}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
-import org.apache.carbondata.core.metadata.encoder.Encoding
-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.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
-import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
-
-private[sql] case class AlterTableAddColumns(
-    alterTableAddColumnsModel: AlterTableAddColumnsModel) extends RunnableCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableAddColumnsModel.tableName
-    val dbName = alterTableAddColumnsModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    var newCols = Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]()
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      // Consider a concurrent scenario where 2 alter operations are executed in parallel. 1st
-      // operation is success and updates the schema file. 2nd operation will get the lock after
-      // completion of 1st operation but as look up relation is called before it will have the
-      // older carbon table and this can lead to inconsistent state in the system. Therefor look
-      // up relation should be called after acquiring the lock
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.carbonTable
-      // get the latest carbon table and check for column existence
-      // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
-      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      val schemaConverter = new ThriftWrapperSchemaConverterImpl()
-      val wrapperTableInfo = schemaConverter
-        .fromExternalToWrapperTableInfo(thriftTableInfo,
-          dbName,
-          tableName,
-          carbonTable.getStorePath)
-      newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
-        dbName,
-        wrapperTableInfo,
-        carbonTablePath,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath, sparkSession.sparkContext).process
-      // generate dictionary files for the newly added columns
-      new AlterTableAddColumnRDD(sparkSession.sparkContext,
-        newCols,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath).collect()
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
-      schemaEvolutionEntry.setTimeStamp(timeStamp)
-      schemaEvolutionEntry.setAdded(newCols.toList.asJava)
-      val thriftTable = schemaConverter
-        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
-          thriftTable)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      LOGGER.info(s"Alter table for add columns is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for add columns is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception =>
-        LOGGER.error(e, "Alter table add columns failed")
-        if (newCols.nonEmpty) {
-          LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
-          new AlterTableDropColumnRDD(sparkSession.sparkContext,
-            newCols,
-            carbonTable.getCarbonTableIdentifier,
-            carbonTable.getStorePath).collect()
-          AlterTableUtil.revertAddColumnChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table add operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}
-
-private[sql] case class AlterTableRenameTable(alterTableRenameModel: AlterTableRenameModel)
-  extends RunnableCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    val oldTableIdentifier = alterTableRenameModel.oldTableIdentifier
-    val newTableIdentifier = alterTableRenameModel.newTableIdentifier
-    val oldDatabaseName = oldTableIdentifier.database
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    val newDatabaseName = newTableIdentifier.database
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    if (!oldDatabaseName.equalsIgnoreCase(newDatabaseName)) {
-      throw new MalformedCarbonCommandException("Database name should be same for both tables")
-    }
-    val tableExists = sparkSession.catalog.tableExists(oldDatabaseName, newTableIdentifier.table)
-    if (tableExists) {
-      throw new MalformedCarbonCommandException(s"Table with name $newTableIdentifier " +
-                                                s"already exists")
-    }
-    val oldTableName = oldTableIdentifier.table.toLowerCase
-    val newTableName = newTableIdentifier.table.toLowerCase
-    LOGGER.audit(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
-    LOGGER.info(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
-    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-    val relation: CarbonRelation =
-      metastore.lookupRelation(oldTableIdentifier.database, oldTableName)(sparkSession)
-        .asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      LOGGER.audit(s"Rename table request has failed. " +
-                   s"Table $oldDatabaseName.$oldTableName does not exist")
-      sys.error(s"Table $oldDatabaseName.$oldTableName does not exist")
-    }
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
-      LockUsage.COMPACTION_LOCK,
-      LockUsage.DELETE_SEGMENT_LOCK,
-      LockUsage.CLEAN_FILES_LOCK,
-      LockUsage.DROP_TABLE_LOCK)
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(oldDatabaseName, oldTableName, locksToBeAcquired)(
-            sparkSession)
-      val tableMeta = metastore.lookupRelation(Some(oldDatabaseName), oldTableName)(sparkSession)
-        .asInstanceOf[CarbonRelation].tableMeta
-      carbonTable = tableMeta.carbonTable
-      // get the latest carbon table and check for column existence
-      val carbonTablePath = CarbonStorePath.
-        getCarbonTablePath(AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath))
-      val tableMetadataFile = carbonTablePath.getPath
-      val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
-      schemaEvolutionEntry.setTableName(newTableName)
-      timeStamp = System.currentTimeMillis()
-      schemaEvolutionEntry.setTime_stamp(timeStamp)
-      renameBadRecords(oldTableName, newTableName, oldDatabaseName)
-      val fileType = FileFactory.getFileType(tableMetadataFile)
-      if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
-        val rename = FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
-          .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
-                       newTableName)
-        if (!rename) {
-          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
-          sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
-        }
-      }
-      val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
-        newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
-      val newTablePath = metastore.updateTableSchema(newTableIdentifier,
-          carbonTable.getCarbonTableIdentifier,
-          tableInfo,
-          schemaEvolutionEntry,
-          tableMeta.tablePath)(sparkSession)
-      metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
-      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
-        .runSqlHive(
-          s"ALTER TABLE $oldDatabaseName.$oldTableName RENAME TO $oldDatabaseName.$newTableName")
-      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
-        .runSqlHive(
-          s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
-          s"('tableName'='$newTableName', " +
-          s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
-      sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
-        Some(oldDatabaseName)).quotedString)
-      LOGGER.audit(s"Table $oldTableName has been successfully renamed to $newTableName")
-      LOGGER.info(s"Table $oldTableName has been successfully renamed to $newTableName")
-    } catch {
-      case e: Exception =>
-        LOGGER.error(e, "Rename table failed: " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil
-            .revertRenameTableChanges(oldTableIdentifier,
-              newTableName,
-              carbonTable.getStorePath,
-              carbonTable.getCarbonTableIdentifier.getTableId,
-              timeStamp)(
-              sparkSession)
-          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
-        }
-        sys.error(s"Alter table rename table operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-      // case specific to rename table as after table rename old table path will not be found
-      if (carbonTable != null) {
-        AlterTableUtil
-          .releaseLocksManually(locks,
-            locksToBeAcquired,
-            oldDatabaseName,
-            newTableName,
-            carbonTable.getStorePath)
-      }
-    }
-    Seq.empty
-  }
-
-  private def renameBadRecords(oldTableName: String,
-      newTableName: String,
-      dataBaseName: String) = {
-    val oldPath = CarbonUtil
-      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + oldTableName)
-    val newPath = CarbonUtil
-      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + newTableName)
-    val fileType = FileFactory.getFileType(oldPath)
-    if (FileFactory.isFileExist(oldPath, fileType)) {
-      val renameSuccess = FileFactory.getCarbonFile(oldPath, fileType)
-        .renameForce(newPath)
-      if (!renameSuccess) {
-        sys.error(s"BadRecords Folder Rename Failed for table $dataBaseName.$oldTableName")
-      }
-    }
-  }
-
-}
-
-private[sql] case class AlterTableDropColumns(
-    alterTableDropColumnModel: AlterTableDropColumnModel) extends RunnableCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableDropColumnModel.tableName
-    val dbName = alterTableDropColumnModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table drop columns request has been received for $dbName.$tableName")
-    var locks = List.empty[ICarbonLock]
-    var timeStamp = 0L
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    // get the latest carbon table and check for column existence
-    var carbonTable: CarbonTable = null
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.carbonTable
-      val partitionInfo = carbonTable.getPartitionInfo(tableName)
-      if (partitionInfo != null) {
-        val partitionColumnSchemaList = partitionInfo.getColumnSchemaList.asScala
-          .map(_.getColumnName)
-        // check each column existence in the table
-        val partitionColumns = alterTableDropColumnModel.columns.filter {
-          tableColumn => partitionColumnSchemaList.contains(tableColumn)
-        }
-        if (partitionColumns.nonEmpty) {
-          throw new UnsupportedOperationException("Partition columns cannot be dropped: " +
-                                                  s"$partitionColumns")
-        }
-      }
-      val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
-      var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column
-      .ColumnSchema]()
-      var keyColumnCountToBeDeleted = 0
-      // TODO: if deleted column list includes bucketted column throw an error
-      alterTableDropColumnModel.columns.foreach { column =>
-        var columnExist = false
-        tableColumns.foreach { tableColumn =>
-          // column should not be already deleted and should exist in the table
-          if (!tableColumn.isInvisible && column.equalsIgnoreCase(tableColumn.getColName)) {
-            if (tableColumn.isDimension) {
-              keyColumnCountToBeDeleted += 1
-              if (tableColumn.hasEncoding(Encoding.DICTIONARY)) {
-                dictionaryColumns ++= Seq(tableColumn.getColumnSchema)
-              }
-            }
-            columnExist = true
-          }
-        }
-        if (!columnExist) {
-          sys.error(s"Column $column does not exists in the table $dbName.$tableName")
-        }
-      }
-      // take the total key column count. key column to be deleted should not
-      // be >= key columns in schema
-      val totalKeyColumnInSchema = tableColumns.count {
-        tableColumn => !tableColumn.isInvisible && tableColumn.isDimension
-      }
-      if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
-        sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
-      }
-      // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
-      val tableInfo: org.apache.carbondata.format.TableInfo =
-        metastore.getThriftTableInfo(carbonTablePath)(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
-      alterTableDropColumnModel.columns.foreach { column =>
-        columnSchemaList.foreach { columnSchema =>
-          if (!columnSchema.invisible && column.equalsIgnoreCase(columnSchema.column_name)) {
-            deletedColumnSchema += columnSchema.deepCopy
-            columnSchema.invisible = true
-          }
-        }
-      }
-      // add deleted columns to schema evolution history and update the schema
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
-      schemaEvolutionEntry.setRemoved(deletedColumnSchema.toList.asJava)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaEvolutionEntry,
-          tableInfo)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      // TODO: 1. add check for deletion of index tables
-      // delete dictionary files for dictionary column and clear dictionary cache from memory
-      new AlterTableDropColumnRDD(sparkSession.sparkContext,
-        dictionaryColumns,
-        carbonTable.getCarbonTableIdentifier,
-        carbonTable.getStorePath).collect()
-      LOGGER.info(s"Alter table for drop columns is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for drop columns is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception => LOGGER
-        .error("Alter table drop columns failed : " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil.revertDropColumnChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table drop column operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}
-
-private[sql] case class AlterTableDataTypeChange(
-    alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel) extends RunnableCommand {
-
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  def run(sparkSession: SparkSession): Seq[Row] = {
-    val tableName = alterTableDataTypeChangeModel.tableName
-    val dbName = alterTableDataTypeChangeModel.databaseName
-      .getOrElse(sparkSession.catalog.currentDatabase)
-    LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
-    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
-    var locks = List.empty[ICarbonLock]
-    // get the latest carbon table and check for column existence
-    var carbonTable: CarbonTable = null
-    var timeStamp = 0L
-    try {
-      locks = AlterTableUtil
-        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
-      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
-      carbonTable = metastore
-        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
-        .tableMeta.carbonTable
-      val columnName = alterTableDataTypeChangeModel.columnName
-      val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala.filter(!_.isInvisible)
-      if (!carbonColumns.exists(_.getColName.equalsIgnoreCase(columnName))) {
-        LOGGER.audit(s"Alter table change data type request has failed. " +
-                     s"Column $columnName does not exist")
-        sys.error(s"Column does not exist: $columnName")
-      }
-      val carbonColumn = carbonColumns.filter(_.getColName.equalsIgnoreCase(columnName))
-      if (carbonColumn.size == 1) {
-        CarbonScalaUtil
-          .validateColumnDataType(alterTableDataTypeChangeModel.dataTypeInfo, carbonColumn(0))
-      } else {
-        LOGGER.audit(s"Alter table change data type request has failed. " +
-                     s"Column $columnName is invalid")
-        sys.error(s"Invalid Column: $columnName")
-      }
-      // read the latest schema file
-      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
-        carbonTable.getCarbonTableIdentifier)
-      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
-      // maintain the added column for schema evolution history
-      var addColumnSchema: ColumnSchema = null
-      var deletedColumnSchema: 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.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
-          columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
-          addColumnSchema = columnSchema
-        }
-      }
-      timeStamp = System.currentTimeMillis
-      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
-      schemaEvolutionEntry.setAdded(List(addColumnSchema).asJava)
-      schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava)
-      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
-        .setTime_stamp(System.currentTimeMillis)
-      AlterTableUtil
-        .updateSchemaInfo(carbonTable,
-          schemaEvolutionEntry,
-          tableInfo)(sparkSession,
-          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
-      LOGGER.info(s"Alter table for data type change is successful for table $dbName.$tableName")
-      LOGGER.audit(s"Alter table for data type change is successful for table $dbName.$tableName")
-    } catch {
-      case e: Exception => LOGGER
-        .error("Alter table change datatype failed : " + e.getMessage)
-        if (carbonTable != null) {
-          AlterTableUtil.revertDataTypeChanges(dbName, tableName, timeStamp)(sparkSession)
-        }
-        sys.error(s"Alter table data type change operation failed: ${e.getMessage}")
-    } finally {
-      // release lock after command execution completion
-      AlterTableUtil.releaseLocks(locks)
-    }
-    Seq.empty
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
new file mode 100644
index 0000000..f5c6cba
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonCreateTableCommand.scala
@@ -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.spark.sql.execution.command
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.exception.InvalidConfigurationException
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.schema.table.TableInfo
+import org.apache.carbondata.core.util.CarbonUtil
+
+case class CarbonCreateTableCommand(
+    cm: TableModel,
+    createDSTable: Boolean = true)
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val storePath = CarbonEnv.getInstance(sparkSession).storePath
+    CarbonEnv.getInstance(sparkSession).carbonMetastore.
+      checkSchemasModifiedTimeAndReloadTables(storePath)
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    cm.databaseName = GetDB.getDatabaseName(cm.databaseNameOp, sparkSession)
+    val tbName = cm.tableName
+    val dbName = cm.databaseName
+    LOGGER.audit(s"Creating Table with Database name [$dbName] and Table name [$tbName]")
+
+    val tableInfo: TableInfo = TableNewProcessor(cm)
+
+    // Add validation for sort scope when create table
+    val sortScope = tableInfo.getFactTable.getTableProperties
+      .getOrDefault("sort_scope", CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT)
+    if (!CarbonUtil.isValidSortOption(sortScope)) {
+      throw new InvalidConfigurationException(
+        s"Passing invalid SORT_SCOPE '$sortScope', valid SORT_SCOPE are 'NO_SORT', 'BATCH_SORT'," +
+        s" 'LOCAL_SORT' and 'GLOBAL_SORT' ")
+    }
+
+    if (tableInfo.getFactTable.getListOfColumns.size <= 0) {
+      sys.error("No Dimensions found. Table should have at least one dimesnion !")
+    }
+
+    if (sparkSession.sessionState.catalog.listTables(dbName)
+      .exists(_.table.equalsIgnoreCase(tbName))) {
+      if (!cm.ifNotExistsSet) {
+        LOGGER.audit(
+          s"Table creation with Database name [$dbName] and Table name [$tbName] failed. " +
+          s"Table [$tbName] already exists under database [$dbName]")
+        sys.error(s"Table [$tbName] already exists under database [$dbName]")
+      }
+    } else {
+      val tableIdentifier = AbsoluteTableIdentifier.from(storePath, dbName, tbName)
+      // Add Database to catalog and persist
+      val catalog = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val tablePath = tableIdentifier.getTablePath
+      val carbonSchemaString = catalog.generateTableSchemaString(tableInfo, tablePath)
+      if (createDSTable) {
+        try {
+          val fields = new Array[Field](cm.dimCols.size + cm.msrCols.size)
+          cm.dimCols.foreach(f => fields(f.schemaOrdinal) = f)
+          cm.msrCols.foreach(f => fields(f.schemaOrdinal) = f)
+
+          sparkSession.sql(
+            s"""CREATE TABLE $dbName.$tbName
+               |(${ fields.map(f => f.rawSchema).mkString(",") })
+               |USING org.apache.spark.sql.CarbonSource""".stripMargin +
+            s""" OPTIONS (tableName "$tbName", dbName "$dbName", tablePath """.stripMargin +
+            s""""$tablePath"$carbonSchemaString) """)
+        } catch {
+          case e: Exception =>
+            val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
+            // call the drop table to delete the created table.
+            CarbonEnv.getInstance(sparkSession).carbonMetastore
+              .dropTable(tablePath, identifier)(sparkSession)
+
+            LOGGER.audit(s"Table creation with Database name [$dbName] " +
+                         s"and Table name [$tbName] failed")
+            throw e
+        }
+      }
+
+      LOGGER.audit(s"Table created with Database name [$dbName] and Table name [$tbName]")
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
new file mode 100644
index 0000000..e5f6b75
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDescribeFormattedCommand.scala
@@ -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.spark.sql.execution.command
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.hive.CarbonRelation
+import org.codehaus.jackson.map.ObjectMapper
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
+
+private[sql] case class CarbonDescribeFormattedCommand(
+    child: SparkPlan,
+    override val output: Seq[Attribute],
+    tblIdentifier: TableIdentifier)
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  private def getColumnGroups(dimensions: List[CarbonDimension]): Seq[(String, String, String)] = {
+    var results: Seq[(String, String, String)] =
+      Seq(("", "", ""), ("##Column Group Information", "", ""))
+    val groupedDimensions = dimensions.groupBy(x => x.columnGroupId()).filter {
+      case (groupId, _) => groupId != -1
+    }.toSeq.sortBy(_._1)
+    val groups = groupedDimensions.map(colGroups => {
+      colGroups._2.map(dim => dim.getColName).mkString(", ")
+    })
+    var index = 1
+    groups.foreach { x =>
+      results = results :+ (s"Column Group $index", x, "")
+      index = index + 1
+    }
+    results
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(tblIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
+    val mapper = new ObjectMapper()
+    val colProps = StringBuilder.newBuilder
+    val dims = relation.metaData.dims.map(x => x.toLowerCase)
+    var results: Seq[(String, String, String)] = child.schema.fields.map { field =>
+      val fieldName = field.name.toLowerCase
+      val comment = if (dims.contains(fieldName)) {
+        val dimension = relation.metaData.carbonTable.getDimensionByName(
+          relation.tableMeta.carbonTableIdentifier.getTableName, fieldName)
+        if (null != dimension.getColumnProperties && !dimension.getColumnProperties.isEmpty) {
+          colProps.append(fieldName).append(".")
+            .append(mapper.writeValueAsString(dimension.getColumnProperties))
+            .append(",")
+        }
+        if (dimension.hasEncoding(Encoding.DICTIONARY) &&
+            !dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          "DICTIONARY, KEY COLUMN" + (if (dimension.hasEncoding(Encoding.INVERTED_INDEX)) {
+            ""
+          } else {
+            ",NOINVERTEDINDEX"
+          })
+        } else {
+          "KEY COLUMN" + (if (dimension.hasEncoding(Encoding.INVERTED_INDEX)) {
+            ""
+          } else {
+            ",NOINVERTEDINDEX"
+          })
+        }
+      } else {
+        "MEASURE"
+      }
+      (field.name, field.dataType.simpleString, comment)
+    }
+    val colPropStr = if (colProps.toString().trim().length() > 0) {
+      // drops additional comma at end
+      colProps.toString().dropRight(1)
+    } else {
+      colProps.toString()
+    }
+    results ++= Seq(("", "", ""), ("##Detailed Table Information", "", ""))
+    results ++= Seq(("Database Name: ", relation.tableMeta.carbonTableIdentifier
+      .getDatabaseName, "")
+    )
+    results ++= Seq(("Table Name: ", relation.tableMeta.carbonTableIdentifier.getTableName, ""))
+    results ++= Seq(("CARBON Store Path: ", relation.tableMeta.storePath, ""))
+    val carbonTable = relation.tableMeta.carbonTable
+    results ++= Seq(("Table Block Size : ", carbonTable.getBlockSizeInMB + " MB", ""))
+    results ++= Seq(("SORT_SCOPE", carbonTable.getTableInfo.getFactTable
+      .getTableProperties.getOrDefault("sort_scope", CarbonCommonConstants
+      .LOAD_SORT_SCOPE_DEFAULT), CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT))
+    results ++= Seq(("", "", ""), ("##Detailed Column property", "", ""))
+    if (colPropStr.length() > 0) {
+      results ++= Seq((colPropStr, "", ""))
+    } else {
+      results ++= Seq(("ADAPTIVE", "", ""))
+    }
+    results ++= Seq(("SORT_COLUMNS", relation.metaData.carbonTable.getSortColumns(
+      relation.tableMeta.carbonTableIdentifier.getTableName).asScala
+      .map(column => column).mkString(","), ""))
+    val dimension = carbonTable
+      .getDimensionByTableName(relation.tableMeta.carbonTableIdentifier.getTableName)
+    results ++= getColumnGroups(dimension.asScala.toList)
+    if (carbonTable.getPartitionInfo(carbonTable.getFactTableName) != null) {
+      results ++=
+      Seq(("Partition Columns: ", carbonTable.getPartitionInfo(carbonTable.getFactTableName)
+        .getColumnSchemaList.asScala.map(_.getColumnName).mkString(","), ""))
+    }
+    results.map { case (name, dataType, comment) =>
+      Row(f"$name%-36s", f"$dataType%-80s", f"$comment%-72s")
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
new file mode 100644
index 0000000..5f70771
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/CarbonDropTableCommand.scala
@@ -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.spark.sql.execution.command
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.{CarbonEnv, GetDB, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.core.util.path.CarbonStorePath
+
+case class CarbonDropTableCommand(
+    ifExistsSet: Boolean,
+    databaseNameOp: Option[String],
+    tableName: String)
+  extends RunnableCommand with SchemaProcessCommand with DataProcessCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+    processData(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val dbName = GetDB.getDatabaseName(databaseNameOp, sparkSession)
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName, "")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.DROP_TABLE_LOCK)
+    val carbonEnv = CarbonEnv.getInstance(sparkSession)
+    val catalog = carbonEnv.carbonMetastore
+    val tableIdentifier =
+      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath,
+        dbName.toLowerCase, tableName.toLowerCase)
+    catalog.checkSchemasModifiedTimeAndReloadTables(tableIdentifier.getStorePath)
+    val carbonLocks: scala.collection.mutable.ListBuffer[ICarbonLock] = ListBuffer()
+    try {
+      locksToBeAcquired foreach {
+        lock => carbonLocks += CarbonLockUtil.getLockObject(carbonTableIdentifier, lock)
+      }
+      LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
+
+      CarbonEnv.getInstance(sparkSession).carbonMetastore
+        .dropTable(tableIdentifier.getTablePath, identifier)(sparkSession)
+      LOGGER.audit(s"Deleted table [$tableName] under database [$dbName]")
+    } catch {
+      case ex: Exception =>
+        LOGGER.error(ex, s"Dropping table $dbName.$tableName failed")
+        sys.error(s"Dropping table $dbName.$tableName failed: ${ex.getMessage}")
+    } finally {
+      if (carbonLocks.nonEmpty) {
+        val unlocked = carbonLocks.forall(_.unlock())
+        if (unlocked) {
+          logInfo("Table MetaData Unlocked Successfully")
+        }
+      }
+    }
+    Seq.empty
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    // delete the table folder
+    val dbName = GetDB.getDatabaseName(databaseNameOp, sparkSession)
+    val tableIdentifier =
+      AbsoluteTableIdentifier.from(CarbonEnv.getInstance(sparkSession).storePath, dbName, tableName)
+    val metadataFilePath =
+      CarbonStorePath.getCarbonTablePath(tableIdentifier).getMetadataDirectoryPath
+    val fileType = FileFactory.getFileType(metadataFilePath)
+    if (FileFactory.isFileExist(metadataFilePath, fileType)) {
+      val file = FileFactory.getCarbonFile(metadataFilePath, fileType)
+      CarbonUtil.deleteFoldersAndFilesSilent(file.getParentFile)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
deleted file mode 100644
index c8c716e..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/DDLStrategy.scala
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.spark.sql.execution.command
-
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
-import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
-
-import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
-
-/**
- * Carbon strategies for ddl commands
- */
-class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
-
-  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
-    plan match {
-      case LoadDataCommand(identifier, path, isLocal, isOverwrite, partition)
-        if CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(identifier)(sparkSession) =>
-        ExecutedCommandExec(LoadTable(identifier.database, identifier.table.toLowerCase, path,
-          Seq(), Map(), isOverwrite)) :: Nil
-      case alter@AlterTableRenameCommand(oldTableIdentifier, newTableIdentifier, _) =>
-        val dbOption = oldTableIdentifier.database.map(_.toLowerCase)
-        val tableIdentifier = TableIdentifier(oldTableIdentifier.table.toLowerCase(), dbOption)
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(tableIdentifier)(
-            sparkSession)
-        if (isCarbonTable) {
-          val renameModel = AlterTableRenameModel(tableIdentifier, newTableIdentifier)
-          ExecutedCommandExec(AlterTableRenameTable(renameModel)) :: Nil
-        } else {
-          ExecutedCommandExec(alter) :: Nil
-        }
-      case DropTableCommand(identifier, ifNotExists, isView, _)
-        if CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .isTablePathExists(identifier)(sparkSession) =>
-        ExecutedCommandExec(
-          CarbonDropTableCommand(ifNotExists, identifier.database,
-            identifier.table.toLowerCase)) :: Nil
-      case ShowLoadsCommand(databaseName, table, limit) =>
-        ExecutedCommandExec(ShowLoads(databaseName, table.toLowerCase, limit, plan.output)) :: Nil
-      case InsertIntoCarbonTable(relation: CarbonDatasourceHadoopRelation,
-      _, child: LogicalPlan, overwrite, _) =>
-        ExecutedCommandExec(LoadTableByInsert(relation, child, overwrite.enabled)) :: Nil
-      case createDb@CreateDatabaseCommand(dbName, ifNotExists, _, _, _) =>
-        CarbonUtil.createDatabaseDirectory(dbName, CarbonEnv.getInstance(sparkSession).storePath)
-        ExecutedCommandExec(createDb) :: Nil
-      case drop@DropDatabaseCommand(dbName, ifExists, isCascade) =>
-        ExecutedCommandExec(CarbonDropDatabaseCommand(drop)) :: Nil
-      case alterTable@AlterTableCompaction(altertablemodel) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(TableIdentifier(altertablemodel.tableName,
-            altertablemodel.dbName))(sparkSession)
-        if (isCarbonTable) {
-          if (altertablemodel.compactionType.equalsIgnoreCase("minor") ||
-              altertablemodel.compactionType.equalsIgnoreCase("major")) {
-            ExecutedCommandExec(alterTable) :: Nil
-          } else {
-            throw new MalformedCarbonCommandException(
-              "Unsupported alter operation on carbon table")
-          }
-        } else {
-          throw new MalformedCarbonCommandException(
-            "Operation not allowed : " + altertablemodel.alterSql)
-        }
-      case dataTypeChange@AlterTableDataTypeChange(alterTableChangeDataTypeModel) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(TableIdentifier(alterTableChangeDataTypeModel.tableName,
-            alterTableChangeDataTypeModel.databaseName))(sparkSession)
-        if (isCarbonTable) {
-          ExecutedCommandExec(dataTypeChange) :: Nil
-        } else {
-          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
-        }
-      case addColumn@AlterTableAddColumns(alterTableAddColumnsModel) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(TableIdentifier(alterTableAddColumnsModel.tableName,
-            alterTableAddColumnsModel.databaseName))(sparkSession)
-        if (isCarbonTable) {
-          ExecutedCommandExec(addColumn) :: Nil
-        } else {
-          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
-        }
-      case dropColumn@AlterTableDropColumns(alterTableDropColumnModel) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(TableIdentifier(alterTableDropColumnModel.tableName,
-            alterTableDropColumnModel.databaseName))(sparkSession)
-        if (isCarbonTable) {
-          ExecutedCommandExec(dropColumn) :: Nil
-        } else {
-          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
-        }
-      case desc@DescribeTableCommand(identifier, partitionSpec, isExtended, isFormatted)
-        if CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(identifier)(sparkSession) && isFormatted =>
-        val resolvedTable =
-          sparkSession.sessionState.executePlan(UnresolvedRelation(identifier, None)).analyzed
-        val resultPlan = sparkSession.sessionState.executePlan(resolvedTable).executedPlan
-        ExecutedCommandExec(DescribeCommandFormatted(resultPlan, plan.output, identifier)) :: Nil
-      case ShowPartitionsCommand(t, cols) =>
-        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
-          .tableExists(t)(sparkSession)
-        if (isCarbonTable) {
-          ExecutedCommandExec(ShowCarbonPartitionsCommand(t)) :: Nil
-        } else {
-          ExecutedCommandExec(ShowPartitionsCommand(t, cols)) :: Nil
-        }
-      case set@SetCommand(kv) =>
-        ExecutedCommandExec(CarbonSetCommand(set)) :: Nil
-      case reset@ResetCommand =>
-        ExecutedCommandExec(CarbonResetCommand()) :: Nil
-      case org.apache.spark.sql.execution.datasources.CreateTable(tableDesc, mode, None)
-        if tableDesc.provider.get != DDLUtils.HIVE_PROVIDER
-           && tableDesc.provider.get.equals("org.apache.spark.sql.CarbonSource") =>
-        val updatedCatalog =
-          CarbonSource.updateCatalogTableWithCarbonSchema(tableDesc, sparkSession)
-        val cmd =
-          CreateDataSourceTableCommand(updatedCatalog, ignoreIfExists = mode == SaveMode.Ignore)
-        ExecutedCommandExec(cmd) :: Nil
-      case _ => Nil
-    }
-  }
-
-}


[41/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
[CARBONDATA-1530] Clean up carbon-processing module

This closes #1391


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

Branch: refs/heads/streaming_ingest
Commit: 349c59c7b0e48d7d7551c7a993233b9f7960da94
Parents: b9f10da
Author: Jacky Li <ja...@qq.com>
Authored: Thu Sep 28 00:26:50 2017 +0800
Committer: QiangCai <qi...@qq.com>
Committed: Sun Oct 1 09:41:19 2017 +0800

----------------------------------------------------------------------
 .../examples/GenerateDictionaryExample.scala    |   2 +-
 .../hadoop/test/util/StoreCreator.java          |  23 +-
 .../carbondata/presto/CarbondataPageSource.java |   4 +-
 .../presto/util/CarbonDataStoreCreator.scala    |  46 +-
 .../dataload/TestLoadDataUseAllDictionary.scala |   3 +-
 .../TestLoadDataWithNotProperInputFile.scala    |   2 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java | 905 -----------------
 .../spark/load/DeleteLoadFolders.java           | 151 ---
 .../carbondata/spark/load/FailureCauses.java    |  28 -
 .../spark/partition/api/DataPartitioner.java    |  35 -
 .../spark/partition/api/Partition.java          |  35 -
 .../partition/api/impl/DefaultLoadBalancer.java |  63 --
 .../api/impl/PartitionMultiFileImpl.java        |  44 -
 .../api/impl/QueryPartitionHelper.java          |  74 --
 .../api/impl/SampleDataPartitionerImpl.java     |  40 -
 .../carbondata/spark/splits/TableSplit.java     | 124 ---
 .../carbondata/spark/util/CarbonQueryUtil.java  | 138 ---
 .../carbondata/spark/util/LoadMetadataUtil.java |  47 -
 .../org/apache/carbondata/spark/util/Util.java  |  30 +
 .../apache/carbondata/spark/CarbonOption.scala  |   2 +-
 .../carbondata/spark/PartitionFactory.scala     |   2 +-
 .../load/DataLoadProcessBuilderOnSpark.scala    |   8 +-
 .../load/DataLoadProcessorStepOnSpark.scala     |  22 +-
 .../spark/load/GlobalSortHelper.scala           |   4 +-
 .../carbondata/spark/load/ValidateUtil.scala    |   4 +-
 .../spark/rdd/AlterTableLoadPartitionRDD.scala  |   8 +-
 .../spark/rdd/CarbonCleanFilesRDD.scala         |   2 +-
 .../spark/rdd/CarbonDeleteLoadByDateRDD.scala   |   2 +-
 .../spark/rdd/CarbonDeleteLoadRDD.scala         |   2 +-
 .../spark/rdd/CarbonDropTableRDD.scala          |   2 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |   9 +-
 .../spark/rdd/CarbonIUDMergerRDD.scala          |   2 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  11 +-
 .../spark/rdd/CarbonScanPartitionRDD.scala      |   7 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   2 +-
 .../spark/rdd/DataManagementFunc.scala          |   6 +-
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  18 +-
 .../carbondata/spark/rdd/UpdateDataLoad.scala   |   6 +-
 .../carbondata/spark/util/CommonUtil.scala      |   9 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  13 +-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |   2 +-
 .../command/carbonTableSchemaCommon.scala       |   7 +-
 .../spark/sql/hive/DistributionUtil.scala       |   2 +-
 .../scala/org/apache/spark/util/FileUtils.scala |   2 +-
 .../org/apache/spark/util/PartitionUtils.scala  |   2 +-
 .../scala/org/apache/spark/util/SparkUtil.scala |   2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  24 +-
 .../sql/execution/command/IUDCommands.scala     |   2 +-
 .../execution/command/carbonTableSchema.scala   |  14 +-
 .../spark/util/AllDictionaryTestCase.scala      |   4 +-
 .../spark/util/DictionaryTestCaseUtil.scala     |   2 +-
 .../util/ExternalColumnDictionaryTestCase.scala |   7 +-
 ...GlobalDictionaryUtilConcurrentTestCase.scala |   4 +-
 .../util/GlobalDictionaryUtilTestCase.scala     |   4 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  24 +-
 .../carbondata/spark/util/QueryPlanUtil.scala   |  55 --
 .../AlterTableCompactionCommand.scala           |   2 +-
 .../command/management/LoadTableCommand.scala   |  10 +-
 .../command/mutation/DeleteExecution.scala      |  30 +-
 .../mutation/ProjectForDeleteCommand.scala      |   2 +-
 .../mutation/ProjectForUpdateCommand.scala      |   2 +-
 .../AlterTableDropCarbonPartitionCommand.scala  |   2 +-
 .../AlterTableSplitCarbonPartitionCommand.scala |   2 +-
 .../spark/util/AllDictionaryTestCase.scala      |   4 +-
 .../spark/util/DictionaryTestCaseUtil.scala     |   2 +-
 .../util/ExternalColumnDictionaryTestCase.scala |   6 +-
 .../api/dataloader/DataLoadModel.java           | 188 ----
 .../processing/api/dataloader/SchemaInfo.java   | 154 ---
 .../constants/DataProcessorConstants.java       |  62 --
 .../constants/TableOptionConstant.java          |  41 -
 .../processing/csvload/BlockDetails.java        |  84 --
 .../processing/csvload/BoundedInputStream.java  | 129 ---
 .../processing/csvload/CSVInputFormat.java      | 326 ------
 .../csvload/CSVRecordReaderIterator.java        | 105 --
 .../processing/csvload/StringArrayWritable.java |  70 --
 .../manager/CarbonDataProcessorManager.java     |  67 --
 .../processing/datatypes/ArrayDataType.java     |   2 +-
 .../processing/datatypes/PrimitiveDataType.java |   6 +-
 .../processing/datatypes/StructDataType.java    |   2 +-
 .../processing/etl/DataLoadingException.java    |  50 -
 .../exception/DataLoadingException.java         |  50 +
 .../exception/SliceMergerException.java         |  78 ++
 .../loading/AbstractDataLoadProcessorStep.java  | 167 ++++
 .../processing/loading/BadRecordsLogger.java    | 278 ++++++
 .../loading/CarbonDataLoadConfiguration.java    | 313 ++++++
 .../processing/loading/DataField.java           |  53 +
 .../processing/loading/DataLoadExecutor.java    | 108 ++
 .../loading/DataLoadProcessBuilder.java         | 226 +++++
 .../processing/loading/FailureCauses.java       |  28 +
 .../loading/complexobjects/ArrayObject.java     |  35 +
 .../loading/complexobjects/StructObject.java    |  36 +
 .../constants/DataLoadProcessorConstants.java   |  39 +
 .../loading/converter/BadRecordLogHolder.java   |  75 ++
 .../converter/DictionaryCardinalityFinder.java  |  26 +
 .../loading/converter/FieldConverter.java       |  36 +
 .../loading/converter/RowConverter.java         |  36 +
 .../AbstractDictionaryFieldConverterImpl.java   |  27 +
 .../impl/ComplexFieldConverterImpl.java         |  58 ++
 .../impl/DictionaryFieldConverterImpl.java      | 134 +++
 .../DirectDictionaryFieldConverterImpl.java     |  88 ++
 .../converter/impl/FieldEncoderFactory.java     | 142 +++
 .../impl/MeasureFieldConverterImpl.java         | 101 ++
 .../impl/NonDictionaryFieldConverterImpl.java   |  90 ++
 .../converter/impl/RowConverterImpl.java        | 241 +++++
 .../loading/csvinput/BlockDetails.java          |  84 ++
 .../loading/csvinput/BoundedInputStream.java    | 129 +++
 .../loading/csvinput/CSVInputFormat.java        | 326 ++++++
 .../csvinput/CSVRecordReaderIterator.java       | 105 ++
 .../loading/csvinput/StringArrayWritable.java   |  70 ++
 .../DictionaryServerClientDictionary.java       |  89 ++
 .../loading/dictionary/DirectDictionary.java    |  57 ++
 .../loading/dictionary/InMemBiDictionary.java   |  80 ++
 .../dictionary/PreCreatedDictionary.java        |  55 ++
 .../exception/BadRecordFoundException.java      |  65 ++
 .../exception/CarbonDataLoadingException.java   |  73 ++
 .../loading/exception/NoRetryException.java     |  68 ++
 .../loading/model/CarbonDataLoadSchema.java     |  57 ++
 .../loading/model/CarbonLoadModel.java          | 764 ++++++++++++++
 .../loading/parser/CarbonParserFactory.java     |  80 ++
 .../loading/parser/ComplexParser.java           |  29 +
 .../loading/parser/GenericParser.java           |  34 +
 .../processing/loading/parser/RowParser.java    |  31 +
 .../loading/parser/impl/ArrayParserImpl.java    |  68 ++
 .../parser/impl/PrimitiveParserImpl.java        |  28 +
 .../loading/parser/impl/RowParserImpl.java      |  95 ++
 .../loading/parser/impl/StructParserImpl.java   |  70 ++
 .../loading/partition/Partitioner.java          |  27 +
 .../partition/impl/HashPartitionerImpl.java     | 106 ++
 .../processing/loading/row/CarbonRowBatch.java  |  64 ++
 .../processing/loading/row/CarbonSortBatch.java |  45 +
 .../loading/sort/AbstractMergeSorter.java       |  43 +
 .../loading/sort/SortScopeOptions.java          |  54 +
 .../loading/sort/SortStepRowUtil.java           |  74 ++
 .../processing/loading/sort/Sorter.java         |  54 +
 .../processing/loading/sort/SorterFactory.java  |  72 ++
 .../sort/impl/ParallelReadMergeSorterImpl.java  | 231 +++++
 ...arallelReadMergeSorterWithBucketingImpl.java | 276 ++++++
 .../loading/sort/impl/ThreadStatusObserver.java |  55 ++
 .../UnsafeBatchParallelReadMergeSorterImpl.java | 338 +++++++
 .../impl/UnsafeParallelReadMergeSorterImpl.java | 216 ++++
 ...arallelReadMergeSorterWithBucketingImpl.java | 266 +++++
 .../sort/unsafe/UnsafeCarbonRowPage.java        | 405 ++++++++
 .../loading/sort/unsafe/UnsafeSortDataRows.java | 413 ++++++++
 .../unsafe/comparator/UnsafeRowComparator.java  | 131 +++
 .../UnsafeRowComparatorForNormalDIms.java       |  59 ++
 .../sort/unsafe/holder/SortTempChunkHolder.java |  36 +
 .../sort/unsafe/holder/UnsafeCarbonRow.java     |  24 +
 .../unsafe/holder/UnsafeCarbonRowForMerge.java  |  23 +
 .../holder/UnsafeFinalMergePageHolder.java      | 105 ++
 .../unsafe/holder/UnsafeInmemoryHolder.java     | 100 ++
 .../holder/UnsafeInmemoryMergeHolder.java       | 105 ++
 .../holder/UnsafeSortTempFileChunkHolder.java   | 472 +++++++++
 .../UnsafeInMemoryIntermediateDataMerger.java   | 213 ++++
 .../merger/UnsafeIntermediateFileMerger.java    | 367 +++++++
 .../unsafe/merger/UnsafeIntermediateMerger.java | 187 ++++
 .../UnsafeSingleThreadFinalSortFilesMerger.java | 259 +++++
 .../loading/sort/unsafe/sort/TimSort.java       | 986 +++++++++++++++++++
 .../unsafe/sort/UnsafeIntSortDataFormat.java    |  72 ++
 .../CarbonRowDataWriterProcessorStepImpl.java   | 299 ++++++
 .../steps/DataConverterProcessorStepImpl.java   | 227 +++++
 ...ConverterProcessorWithBucketingStepImpl.java | 231 +++++
 .../steps/DataWriterBatchProcessorStepImpl.java | 155 +++
 .../steps/DataWriterProcessorStepImpl.java      | 199 ++++
 .../loading/steps/InputProcessorStepImpl.java   | 244 +++++
 .../loading/steps/SortProcessorStepImpl.java    |  83 ++
 .../merger/AbstractResultProcessor.java         |   2 +-
 .../processing/merger/CarbonDataMergerUtil.java |   2 +-
 .../merger/CompactionResultSortProcessor.java   |  12 +-
 .../merger/RowResultMergerProcessor.java        |   4 +-
 .../merger/exeception/SliceMergerException.java |  78 --
 .../processing/model/CarbonDataLoadSchema.java  |  57 --
 .../processing/model/CarbonLoadModel.java       | 764 --------------
 .../newflow/AbstractDataLoadProcessorStep.java  | 167 ----
 .../newflow/CarbonDataLoadConfiguration.java    | 313 ------
 .../processing/newflow/DataField.java           |  53 -
 .../processing/newflow/DataLoadExecutor.java    | 109 --
 .../newflow/DataLoadProcessBuilder.java         | 231 -----
 .../newflow/complexobjects/ArrayObject.java     |  35 -
 .../newflow/complexobjects/StructObject.java    |  36 -
 .../constants/DataLoadProcessorConstants.java   |  39 -
 .../newflow/converter/BadRecordLogHolder.java   |  75 --
 .../converter/DictionaryCardinalityFinder.java  |  26 -
 .../newflow/converter/FieldConverter.java       |  36 -
 .../newflow/converter/RowConverter.java         |  36 -
 .../AbstractDictionaryFieldConverterImpl.java   |  27 -
 .../impl/ComplexFieldConverterImpl.java         |  58 --
 .../impl/DictionaryFieldConverterImpl.java      | 134 ---
 .../DirectDictionaryFieldConverterImpl.java     |  88 --
 .../converter/impl/FieldEncoderFactory.java     | 142 ---
 .../impl/MeasureFieldConverterImpl.java         | 101 --
 .../impl/NonDictionaryFieldConverterImpl.java   |  90 --
 .../converter/impl/RowConverterImpl.java        | 241 -----
 .../DictionaryServerClientDictionary.java       |  89 --
 .../newflow/dictionary/DirectDictionary.java    |  57 --
 .../newflow/dictionary/InMemBiDictionary.java   |  80 --
 .../dictionary/PreCreatedDictionary.java        |  55 --
 .../exception/BadRecordFoundException.java      |  65 --
 .../exception/CarbonDataLoadingException.java   |  73 --
 .../newflow/exception/NoRetryException.java     |  68 --
 .../newflow/parser/CarbonParserFactory.java     |  80 --
 .../newflow/parser/ComplexParser.java           |  29 -
 .../newflow/parser/GenericParser.java           |  34 -
 .../processing/newflow/parser/RowParser.java    |  31 -
 .../newflow/parser/impl/ArrayParserImpl.java    |  68 --
 .../parser/impl/PrimitiveParserImpl.java        |  28 -
 .../newflow/parser/impl/RowParserImpl.java      |  95 --
 .../newflow/parser/impl/StructParserImpl.java   |  70 --
 .../newflow/partition/Partitioner.java          |  27 -
 .../partition/impl/HashPartitionerImpl.java     | 106 --
 .../processing/newflow/row/CarbonRowBatch.java  |  64 --
 .../processing/newflow/row/CarbonSortBatch.java |  45 -
 .../newflow/sort/AbstractMergeSorter.java       |  43 -
 .../newflow/sort/SortScopeOptions.java          |  54 -
 .../newflow/sort/SortStepRowUtil.java           |  74 --
 .../processing/newflow/sort/Sorter.java         |  54 -
 .../processing/newflow/sort/SorterFactory.java  |  72 --
 .../sort/impl/ParallelReadMergeSorterImpl.java  | 231 -----
 ...arallelReadMergeSorterWithBucketingImpl.java | 276 ------
 .../newflow/sort/impl/ThreadStatusObserver.java |  55 --
 .../UnsafeBatchParallelReadMergeSorterImpl.java | 338 -------
 .../impl/UnsafeParallelReadMergeSorterImpl.java | 216 ----
 ...arallelReadMergeSorterWithBucketingImpl.java | 266 -----
 .../sort/unsafe/UnsafeCarbonRowPage.java        | 405 --------
 .../newflow/sort/unsafe/UnsafeSortDataRows.java | 413 --------
 .../unsafe/comparator/UnsafeRowComparator.java  | 131 ---
 .../UnsafeRowComparatorForNormalDIms.java       |  59 --
 .../sort/unsafe/holder/SortTempChunkHolder.java |  36 -
 .../sort/unsafe/holder/UnsafeCarbonRow.java     |  24 -
 .../unsafe/holder/UnsafeCarbonRowForMerge.java  |  23 -
 .../holder/UnsafeFinalMergePageHolder.java      | 105 --
 .../unsafe/holder/UnsafeInmemoryHolder.java     | 100 --
 .../holder/UnsafeInmemoryMergeHolder.java       | 105 --
 .../holder/UnsafeSortTempFileChunkHolder.java   | 472 ---------
 .../UnsafeInMemoryIntermediateDataMerger.java   | 213 ----
 .../merger/UnsafeIntermediateFileMerger.java    | 367 -------
 .../unsafe/merger/UnsafeIntermediateMerger.java | 187 ----
 .../UnsafeSingleThreadFinalSortFilesMerger.java | 259 -----
 .../newflow/sort/unsafe/sort/TimSort.java       | 986 -------------------
 .../unsafe/sort/UnsafeIntSortDataFormat.java    |  72 --
 .../CarbonRowDataWriterProcessorStepImpl.java   | 299 ------
 .../steps/DataConverterProcessorStepImpl.java   | 227 -----
 ...ConverterProcessorWithBucketingStepImpl.java | 231 -----
 .../steps/DataWriterBatchProcessorStepImpl.java | 155 ---
 .../steps/DataWriterProcessorStepImpl.java      | 199 ----
 .../newflow/steps/InputProcessorStepImpl.java   | 244 -----
 .../newflow/steps/SortProcessorStepImpl.java    |  83 --
 .../processing/partition/DataPartitioner.java   |  35 +
 .../processing/partition/Partition.java         |  35 +
 .../partition/impl/DefaultLoadBalancer.java     |  63 ++
 .../partition/impl/PartitionMultiFileImpl.java  |  44 +
 .../partition/impl/QueryPartitionHelper.java    |  74 ++
 .../impl/SampleDataPartitionerImpl.java         |  40 +
 .../spliter/AbstractCarbonQueryExecutor.java    | 133 +++
 .../partition/spliter/CarbonSplitExecutor.java  |  64 ++
 .../partition/spliter/RowResultProcessor.java   | 105 ++
 .../exception/AlterPartitionSliceException.java |  78 ++
 .../schema/metadata/SortObserver.java           |  42 -
 .../CarbonSortKeyAndGroupByException.java       |  87 ++
 .../sortdata/AbstractTempSortFileWriter.java    | 100 ++
 .../sortdata/CompressedTempSortFileWriter.java  |  78 ++
 .../sort/sortdata/IntermediateFileMerger.java   | 385 ++++++++
 .../sort/sortdata/NewRowComparator.java         |  71 ++
 .../sortdata/NewRowComparatorForNormalDims.java |  59 ++
 .../processing/sort/sortdata/RowComparator.java |  94 ++
 .../sortdata/RowComparatorForNormalDims.java    |  62 ++
 .../SingleThreadFinalSortFilesMerger.java       | 312 ++++++
 .../processing/sort/sortdata/SortDataRows.java  | 437 ++++++++
 .../sortdata/SortIntermediateFileMerger.java    | 114 +++
 .../processing/sort/sortdata/SortObserver.java  |  42 +
 .../sort/sortdata/SortParameters.java           | 602 +++++++++++
 .../sort/sortdata/SortTempFileChunkHolder.java  | 522 ++++++++++
 .../sort/sortdata/SortTempFileChunkWriter.java  |  75 ++
 .../sort/sortdata/TempSortFileReader.java       |  37 +
 .../sort/sortdata/TempSortFileWriter.java       |  46 +
 .../sortdata/TempSortFileWriterFactory.java     |  41 +
 .../UnCompressedTempSortFileWriter.java         | 112 +++
 .../CarbonSortKeyAndGroupByException.java       |  87 --
 .../sortdata/AbstractTempSortFileWriter.java    | 100 --
 .../sortdata/CompressedTempSortFileWriter.java  |  78 --
 .../sortdata/IntermediateFileMerger.java        | 385 --------
 .../sortdata/NewRowComparator.java              |  71 --
 .../sortdata/NewRowComparatorForNormalDims.java |  59 --
 .../sortandgroupby/sortdata/RowComparator.java  |  94 --
 .../sortdata/RowComparatorForNormalDims.java    |  62 --
 .../sortandgroupby/sortdata/SortDataRows.java   | 437 --------
 .../sortdata/SortIntermediateFileMerger.java    | 114 ---
 .../sortandgroupby/sortdata/SortParameters.java | 603 ------------
 .../sortdata/SortTempFileChunkHolder.java       | 522 ----------
 .../sortdata/SortTempFileChunkWriter.java       |  75 --
 .../sortdata/TempSortFileReader.java            |  37 -
 .../sortdata/TempSortFileWriter.java            |  46 -
 .../sortdata/TempSortFileWriterFactory.java     |  41 -
 .../UnCompressedTempSortFileWriter.java         | 112 ---
 .../spliter/AbstractCarbonQueryExecutor.java    | 133 ---
 .../processing/spliter/CarbonSplitExecutor.java |  64 --
 .../processing/spliter/RowResultProcessor.java  | 105 --
 .../exception/AlterPartitionSliceException.java |  78 --
 .../processing/splits/TableSplit.java           | 124 +++
 .../store/CarbonDataFileAttributes.java         |   9 -
 .../store/CarbonDataWriterFactory.java          |   4 +-
 .../store/CarbonFactDataHandlerColumnar.java    |  22 +-
 .../store/CarbonFactDataHandlerModel.java       |   8 +-
 .../processing/store/CarbonKeyBlockHolder.java  |  46 -
 .../store/SingleThreadFinalSortFilesMerger.java | 313 ------
 .../store/colgroup/ColGroupDataHolder.java      | 100 --
 .../store/colgroup/ColGroupMinMax.java          | 215 ----
 .../store/colgroup/ColumnDataHolder.java        |  38 -
 .../processing/store/colgroup/DataHolder.java   |  38 -
 .../store/writer/AbstractFactDataWriter.java    |   2 +-
 .../store/writer/CarbonFactDataWriter.java      |   2 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |   2 +-
 .../csvbased/BadRecordsLogger.java              | 278 ------
 .../util/CarbonDataProcessorUtil.java           |   8 +-
 .../processing/util/CarbonLoaderUtil.java       | 890 +++++++++++++++++
 .../processing/util/CarbonQueryUtil.java        | 124 +++
 .../processing/util/DeleteLoadFolders.java      | 151 +++
 .../processing/util/LoadMetadataUtil.java       |  47 +
 .../processing/util/TableOptionConstant.java    |  41 +
 .../carbondata/processing/StoreCreator.java     |  23 +-
 .../processing/csvload/CSVInputFormatTest.java  | 169 ----
 .../loading/csvinput/CSVInputFormatTest.java    | 169 ++++
 .../dictionary/InMemBiDictionaryTest.java       |  72 ++
 .../dictionary/InMemBiDictionaryTest.java       |  72 --
 .../store/colgroup/ColGroupMinMaxTest.java      | 227 -----
 324 files changed, 18066 insertions(+), 19292 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala b/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala
index 8b2ceba..94d35b0 100644
--- a/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala
+++ b/examples/spark/src/main/scala/org/apache/carbondata/examples/GenerateDictionaryExample.scala
@@ -25,7 +25,7 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension
 import org.apache.carbondata.core.util.path.CarbonStorePath
 import org.apache.carbondata.examples.util.ExampleUtils
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
 
 /**
  * example for global dictionary generation

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 9be3ed5..4b04116 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
@@ -72,16 +72,15 @@ import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWrit
 import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
 import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo;
 import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfoPreparator;
-import org.apache.carbondata.processing.api.dataloader.SchemaInfo;
-import org.apache.carbondata.processing.constants.TableOptionConstant;
-import org.apache.carbondata.processing.csvload.BlockDetails;
-import org.apache.carbondata.processing.csvload.CSVInputFormat;
-import org.apache.carbondata.processing.csvload.CSVRecordReaderIterator;
-import org.apache.carbondata.processing.csvload.StringArrayWritable;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
-import org.apache.carbondata.processing.newflow.DataLoadExecutor;
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.util.TableOptionConstant;
+import org.apache.carbondata.processing.loading.csvinput.BlockDetails;
+import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
+import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator;
+import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable;
+import org.apache.carbondata.processing.loading.model.CarbonDataLoadSchema;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.loading.DataLoadExecutor;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
 
 import com.google.gson.Gson;
 import org.apache.hadoop.conf.Configuration;
@@ -394,7 +393,6 @@ public class StoreCreator {
       path.delete();
     }
 
-    SchemaInfo info = new SchemaInfo();
     BlockDetails blockDetails = new BlockDetails(new Path(loadModel.getFactFilePath()),
         0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"});
     Configuration configuration = new Configuration();
@@ -420,9 +418,6 @@ public class StoreCreator {
         new String[] {storeLocation},
         new CarbonIterator[]{readerIterator});
 
-    info.setDatabaseName(databaseName);
-    info.setTableName(tableName);
-
     writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(),
         new ArrayList<LoadMetadataDetails>());
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 4520476..fd65230 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
@@ -21,13 +21,11 @@ import java.io.IOException;
 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.scan.result.BatchResult;
 import org.apache.carbondata.presto.readers.StreamReader;
 import org.apache.carbondata.presto.readers.StreamReaders;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
 
 import com.facebook.presto.hadoop.$internal.com.google.common.base.Throwables;
 import com.facebook.presto.spi.ConnectorPageSource;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 64f892d..07f9699 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
@@ -17,10 +17,10 @@
 
 package org.apache.carbondata.presto.util
 
-import java.util
 import java.io._
 import java.nio.charset.Charset
 import java.text.SimpleDateFormat
+import java.util
 import java.util.{ArrayList, Date, List, UUID}
 
 import scala.collection.JavaConversions._
@@ -30,42 +30,33 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hadoop.io.NullWritable
 import org.apache.hadoop.mapred.TaskAttemptID
-import org.apache.hadoop.mapreduce.{RecordReader, TaskType}
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.hadoop.mapreduce.{RecordReader, TaskType}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier, ReverseDictionary}
 import org.apache.carbondata.core.cache.{Cache, CacheProvider, CacheType}
-import org.apache.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier,
-ReverseDictionary}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.impl.FileFactory
-import org.apache.carbondata.core.fileoperations.{AtomicFileOperations, AtomicFileOperationsImpl,
-FileWriteOperation}
-import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata,
-CarbonTableIdentifier, ColumnIdentifier}
-import org.apache.carbondata.core.metadata.converter.{SchemaConverter,
-ThriftWrapperSchemaConverterImpl}
+import org.apache.carbondata.core.fileoperations.{AtomicFileOperations, AtomicFileOperationsImpl, FileWriteOperation}
+import org.apache.carbondata.core.metadata.converter.{SchemaConverter, ThriftWrapperSchemaConverterImpl}
 import org.apache.carbondata.core.metadata.datatype.DataType
 import org.apache.carbondata.core.metadata.encoder.Encoding
-import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolutionEntry}
+import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension, CarbonMeasure, ColumnSchema}
 import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
-import org.apache.carbondata.core.metadata.schema.table.column.{CarbonColumn, CarbonDimension,
-CarbonMeasure, ColumnSchema}
+import org.apache.carbondata.core.metadata.schema.{SchemaEvolution, SchemaEvolutionEntry}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonMetadata, CarbonTableIdentifier, ColumnIdentifier}
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
-import org.apache.carbondata.core.writer.{CarbonDictionaryWriter, CarbonDictionaryWriterImpl,
-ThriftWriter}
-import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter,
-CarbonDictionarySortIndexWriterImpl, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
-import org.apache.carbondata.processing.api.dataloader.SchemaInfo
-import org.apache.carbondata.processing.constants.TableOptionConstant
-import org.apache.carbondata.processing.csvload.{BlockDetails, CSVInputFormat,
-CSVRecordReaderIterator, StringArrayWritable}
-import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
-import org.apache.carbondata.processing.newflow.DataLoadExecutor
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.writer.sortindex.{CarbonDictionarySortIndexWriter, CarbonDictionarySortIndexWriterImpl, CarbonDictionarySortInfo, CarbonDictionarySortInfoPreparator}
+import org.apache.carbondata.core.writer.{CarbonDictionaryWriter, CarbonDictionaryWriterImpl, ThriftWriter}
+import org.apache.carbondata.processing.loading.csvinput.{BlockDetails, CSVInputFormat, CSVRecordReaderIterator, StringArrayWritable}
+import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.processing.loading.DataLoadExecutor
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException
+import org.apache.carbondata.processing.util.TableOptionConstant
 
 object CarbonDataStoreCreator {
 
@@ -455,7 +446,6 @@ object CarbonDataStoreCreator {
     if (path.exists()) {
       path.delete()
     }
-    val info: SchemaInfo = new SchemaInfo()
     val blockDetails: BlockDetails = new BlockDetails(
       new Path(loadModel.getFactFilePath),
       0,
@@ -488,8 +478,6 @@ object CarbonDataStoreCreator {
       hadoopAttemptContext)
     new DataLoadExecutor()
       .execute(loadModel, Array(storeLocation), Array(readerIterator))
-    info.setDatabaseName(databaseName)
-    info.setTableName(tableName)
     writeLoadMetadata(loadModel.getCarbonDataLoadSchema,
       loadModel.getTableName,
       loadModel.getTableName,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
index 8e16ba9..52edf3a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
@@ -18,9 +18,10 @@
 package org.apache.carbondata.spark.testsuite.dataload
 
 import org.scalatest.BeforeAndAfterAll
-import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.spark.sql.test.util.QueryTest
 
+import org.apache.carbondata.processing.exception.DataLoadingException
+
 class TestLoadDataUseAllDictionary extends QueryTest with BeforeAndAfterAll{
   override def beforeAll {
     sql("DROP TABLE IF EXISTS t3")

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNotProperInputFile.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNotProperInputFile.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNotProperInputFile.scala
index 016b195..9237627 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNotProperInputFile.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithNotProperInputFile.scala
@@ -20,7 +20,7 @@ package org.apache.carbondata.spark.testsuite.dataload
 import java.io.File
 
 import org.apache.spark.util.FileUtils
-import org.apache.carbondata.processing.model.CarbonLoadModel
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel
 import org.apache.carbondata.spark.util.GlobalDictionaryUtil
 import org.apache.spark.sql.test.util.QueryTest
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
deleted file mode 100644
index 9fe003f..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
+++ /dev/null
@@ -1,905 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.io.BufferedWriter;
-import java.io.DataOutputStream;
-import java.io.File;
-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.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.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.block.Distributable;
-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.datastore.row.LoadStatusType;
-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.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;
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
-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.merger.NodeBlockRelation;
-import org.apache.carbondata.processing.merger.NodeMultiBlockRelation;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import com.google.gson.Gson;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.spark.SparkConf;
-import org.apache.spark.util.Utils;
-
-public final class CarbonLoaderUtil {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonLoaderUtil.class.getName());
-
-  private CarbonLoaderUtil() {
-  }
-
-
-
-  public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
-    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
-
-    for (int i = 0; i < carbonTable.getPartitionCount(); i++) {
-      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(i + "", currentLoad + "");
-      deleteStorePath(segmentPath);
-    }
-  }
-
-  /**
-   * the method returns true if the segment has carbondata file else returns false.
-   *
-   * @param loadModel
-   * @param currentLoad
-   * @return
-   */
-  public static boolean isValidSegment(CarbonLoadModel loadModel,
-      int currentLoad) {
-    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema()
-        .getCarbonTable();
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
-        loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
-
-    int fileCount = 0;
-    int partitionCount = carbonTable.getPartitionCount();
-    for (int i = 0; i < partitionCount; i++) {
-      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(i + "",
-          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;
-    }
-    return true;
-  }
-  public static void deletePartialLoadDataIfExist(CarbonLoadModel loadModel,
-      final boolean isCompactionFlow) throws IOException {
-    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
-    String metaDataLocation = carbonTable.getMetaDataFilepath();
-    final LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metaDataLocation);
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
-
-    //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);
-      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) && details[j].getPartitionCount()
-                  .equals(partitionCount)) {
-                found = true;
-                break;
-              }
-            }
-            return !found;
-          }
-        });
-        for (int k = 0; k < listFiles.length; k++) {
-          String segmentId =
-              CarbonTablePath.DataPathUtil.getSegmentId(listFiles[k].getAbsolutePath() + "/dummy");
-          if (isCompactionFlow) {
-            if (segmentId.contains(".")) {
-              deleteStorePath(listFiles[k].getAbsolutePath());
-            }
-          } else {
-            if (!segmentId.contains(".")) {
-              deleteStorePath(listFiles[k].getAbsolutePath());
-            }
-          }
-        }
-      }
-    }
-  }
-
-  private static void deleteStorePath(String path) {
-    try {
-      FileType fileType = FileFactory.getFileType(path);
-      if (FileFactory.isFileExist(path, fileType)) {
-        CarbonFile carbonFile = FileFactory.getCarbonFile(path, fileType);
-        CarbonUtil.deleteFoldersAndFiles(carbonFile);
-      }
-    } catch (IOException | InterruptedException e) {
-      LOGGER.error("Unable to delete the given path :: " + e.getMessage());
-    }
-  }
-
-
-  /**
-   * This method will delete the local data load folder location after data load is complete
-   *
-   * @param loadModel
-   */
-  public static void deleteLocalDataLoadFolderLocation(CarbonLoadModel loadModel,
-      boolean isCompactionFlow, boolean isAltPartitionFlow) {
-    String databaseName = loadModel.getDatabaseName();
-    String tableName = loadModel.getTableName();
-    String tempLocationKey = CarbonDataProcessorUtil
-        .getTempStoreLocationKey(databaseName, tableName, loadModel.getSegmentId(),
-            loadModel.getTaskNo(), isCompactionFlow, isAltPartitionFlow);
-    // form local store location
-    final String localStoreLocations = CarbonProperties.getInstance().getProperty(tempLocationKey);
-    if (localStoreLocations == null) {
-      throw new RuntimeException("Store location not set for the key " + tempLocationKey);
-    }
-    // submit local folder clean up in another thread so that main thread execution is not blocked
-    ExecutorService localFolderDeletionService = Executors.newFixedThreadPool(1);
-    try {
-      localFolderDeletionService.submit(new Callable<Void>() {
-        @Override public Void call() throws Exception {
-          long startTime = System.currentTimeMillis();
-          String[] locArray = StringUtils.split(localStoreLocations, File.pathSeparator);
-          for (String loc : locArray) {
-            try {
-              CarbonUtil.deleteFoldersAndFiles(new File(loc));
-            } catch (IOException | InterruptedException e) {
-              LOGGER.error(e,
-                  "Failed to delete local data load folder location: " + loc);
-            }
-          }
-          LOGGER.info("Deleted the local store location: " + localStoreLocations
-                + " : Time taken: " + (System.currentTimeMillis() - startTime));
-          return null;
-        }
-      });
-    } finally {
-      if (null != localFolderDeletionService) {
-        localFolderDeletionService.shutdown();
-      }
-    }
-
-  }
-
-  /**
-   * This API will write the load level metadata for the loadmanagement module inorder to
-   * manage the load and query execution management smoothly.
-   *
-   * @param newMetaEntry
-   * @param loadModel
-   * @return boolean which determines whether status update is done or not.
-   * @throws IOException
-   */
-  public static boolean recordLoadMetadata(LoadMetadataDetails newMetaEntry,
-      CarbonLoadModel loadModel, boolean loadStartEntry, boolean insertOverwrite)
-      throws IOException, InterruptedException {
-    boolean status = false;
-    String metaDataFilepath =
-        loadModel.getCarbonDataLoadSchema().getCarbonTable().getMetaDataFilepath();
-    AbsoluteTableIdentifier absoluteTableIdentifier =
-        loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    String tableStatusPath = carbonTablePath.getTableStatusFilePath();
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
-    ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
-    try {
-      if (carbonLock.lockWithRetries()) {
-        LOGGER.info(
-            "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
-                + " for table status updation");
-        LoadMetadataDetails[] listOfLoadFolderDetailsArray =
-            SegmentStatusManager.readLoadMetadata(metaDataFilepath);
-        List<LoadMetadataDetails> listOfLoadFolderDetails =
-            new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        List<CarbonFile> staleFolders = new ArrayList<>();
-        Collections.addAll(listOfLoadFolderDetails, listOfLoadFolderDetailsArray);
-        // create a new segment Id if load has just begun else add the already generated Id
-        if (loadStartEntry) {
-          String segmentId =
-              String.valueOf(SegmentStatusManager.createNewSegmentId(listOfLoadFolderDetailsArray));
-          newMetaEntry.setLoadName(segmentId);
-          loadModel.setLoadMetadataDetails(listOfLoadFolderDetails);
-          loadModel.setSegmentId(segmentId);
-          // Exception should be thrown if:
-          // 1. If insert overwrite is in progress and any other load or insert operation
-          // is triggered
-          // 2. If load or insert into operation is in progress and insert overwrite operation
-          // is triggered
-          for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
-            if (entry.getLoadStatus().equals(LoadStatusType.INSERT_OVERWRITE.getMessage())) {
-              throw new RuntimeException("Already insert overwrite is in progress");
-            } else if (
-                newMetaEntry.getLoadStatus().equals(LoadStatusType.INSERT_OVERWRITE.getMessage())
-                    && entry.getLoadStatus().equals(LoadStatusType.IN_PROGRESS.getMessage())) {
-              throw new RuntimeException("Already insert into or load is in progress");
-            }
-          }
-          listOfLoadFolderDetails.add(newMetaEntry);
-        } else {
-          newMetaEntry.setLoadName(String.valueOf(loadModel.getSegmentId()));
-          // existing entry needs to be overwritten as the entry will exist with some
-          // intermediate status
-          int indexToOverwriteNewMetaEntry = 0;
-          for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
-            if (entry.getLoadName().equals(newMetaEntry.getLoadName())
-                && entry.getLoadStartTime() == newMetaEntry.getLoadStartTime()) {
-              break;
-            }
-            indexToOverwriteNewMetaEntry++;
-          }
-          if (listOfLoadFolderDetails.get(indexToOverwriteNewMetaEntry).getLoadStatus()
-              .equals(CarbonCommonConstants.MARKED_FOR_DELETE)) {
-            throw new RuntimeException("It seems insert overwrite has been issued during load");
-          }
-          if (insertOverwrite) {
-            for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
-              if (!entry.getLoadStatus().equals(LoadStatusType.INSERT_OVERWRITE.getMessage())) {
-                entry.setLoadStatus(CarbonCommonConstants.MARKED_FOR_DELETE);
-                // For insert overwrite, we will delete the old segment folder immediately
-                // So collect the old segments here
-                String path = carbonTablePath.getCarbonDataDirectoryPath("0", 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))) {
-                  staleFolders.add(FileFactory.getCarbonFile(path));
-                }
-              }
-            }
-          }
-          listOfLoadFolderDetails.set(indexToOverwriteNewMetaEntry, newMetaEntry);
-        }
-        SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetails
-            .toArray(new LoadMetadataDetails[listOfLoadFolderDetails.size()]));
-        // Delete all old stale segment folders
-        for (CarbonFile staleFolder : staleFolders) {
-          // try block is inside for loop because even if there is failure in deletion of 1 stale
-          // folder still remaining stale folders should be deleted
-          try {
-            CarbonUtil.deleteFoldersAndFiles(staleFolder);
-          } catch (IOException | InterruptedException e) {
-            LOGGER.error("Failed to delete stale folder: " + e.getMessage());
-          }
-        }
-        status = true;
-      } else {
-        LOGGER.error("Not able to acquire the lock for Table status updation for table " + loadModel
-            .getDatabaseName() + "." + loadModel.getTableName());
-      };
-    } finally {
-      if (carbonLock.unlock()) {
-        LOGGER.info(
-            "Table unlocked successfully after table status updation" + loadModel.getDatabaseName()
-                + "." + loadModel.getTableName());
-      } else {
-        LOGGER.error(
-            "Unable to unlock Table lock for table" + loadModel.getDatabaseName() + "." + loadModel
-                .getTableName() + " during table status updation");
-      }
-    }
-    return status;
-  }
-
-  /**
-   * Method to create new entry for load in table status file
-   *
-   * @param loadMetadataDetails
-   * @param loadStatus
-   * @param loadStartTime
-   * @param addLoadEndTime
-   */
-  public static void populateNewLoadMetaEntry(LoadMetadataDetails loadMetadataDetails,
-      String loadStatus, long loadStartTime, boolean addLoadEndTime) {
-    if (addLoadEndTime) {
-      long loadEndDate = CarbonUpdateUtil.readCurrentTime();
-      loadMetadataDetails.setLoadEndTime(loadEndDate);
-    }
-    loadMetadataDetails.setLoadStatus(loadStatus);
-    loadMetadataDetails.setLoadStartTime(loadStartTime);
-  }
-
-  public static void writeLoadMetadata(String storeLocation, String dbName, String tableName,
-      List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(storeLocation, dbName, tableName);
-    String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
-
-    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 Dictionary getDictionary(DictionaryColumnUniqueIdentifier columnIdentifier,
-      String carbonStorePath) throws IOException {
-    Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
-        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY, carbonStorePath);
-    return dictCache.get(columnIdentifier);
-  }
-
-  public static Dictionary getDictionary(CarbonTableIdentifier tableIdentifier,
-      ColumnIdentifier columnIdentifier, String carbonStorePath, DataType dataType)
-      throws IOException {
-    return getDictionary(
-        new DictionaryColumnUniqueIdentifier(tableIdentifier, columnIdentifier, dataType,
-            CarbonStorePath.getCarbonTablePath(carbonStorePath, tableIdentifier)),
-        carbonStorePath);
-  }
-
-  /**
-   * This method will divide the blocks among the tasks of the nodes as per the data locality
-   *
-   * @param blockInfos
-   * @param noOfNodesInput -1 if number of nodes has to be decided
-   *                       based on block location information
-   * @param parallelism    total no of tasks to execute in parallel
-   * @return
-   */
-  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);
-    int taskPerNode = parallelism / mapOfNodes.size();
-    //assigning non zero value to noOfTasksPerNode
-    int noOfTasksPerNode = taskPerNode == 0 ? 1 : taskPerNode;
-    // divide the blocks of a node among the tasks of the node.
-    return assignBlocksToTasksPerNode(mapOfNodes, noOfTasksPerNode);
-  }
-
-  /**
-   * 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,
-      int noOfNodesInput) {
-    return nodeBlockMapping(blockInfos, noOfNodesInput, null);
-  }
-
-  /**
-   * 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
-   * @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 noOfNodesInput -1 if number of nodes has to be decided
-   *                       based on block location information
-   * @return
-   */
-  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;
-
-    // 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);
-
-    // 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(nodeBlocksMap, uniqueBlocks, blocksPerNode, activeNodes);
-
-    return nodeBlocksMap;
-  }
-
-  /**
-   * Assigning the blocks of a node to tasks.
-   *
-   * @param nodeBlocksMap nodeName to list of blocks mapping
-   * @param noOfTasksPerNode
-   * @return
-   */
-  private static Map<String, List<List<Distributable>>> assignBlocksToTasksPerNode(
-      Map<String, List<Distributable>> nodeBlocksMap, int noOfTasksPerNode) {
-    Map<String, List<List<Distributable>>> outputMap =
-        new HashMap<String, List<List<Distributable>>>(
-            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    // for each node
-    for (Map.Entry<String, List<Distributable>> eachNode : nodeBlocksMap.entrySet()) {
-
-      List<Distributable> blockOfEachNode = eachNode.getValue();
-      //sorting the block so same block will be give to same executor
-      Collections.sort(blockOfEachNode);
-      // create the task list for each node.
-      createTaskListForNode(outputMap, noOfTasksPerNode, eachNode.getKey());
-
-      // take all the block of node and divide it among the tasks of a node.
-      divideBlockToTasks(outputMap, eachNode.getKey(), blockOfEachNode);
-    }
-
-    return outputMap;
-  }
-
-  /**
-   * This will divide the blocks of a node to tasks of the node.
-   *
-   * @param outputMap
-   * @param key
-   * @param blockOfEachNode
-   */
-  private static void divideBlockToTasks(Map<String, List<List<Distributable>>> outputMap,
-      String key, List<Distributable> blockOfEachNode) {
-
-    List<List<Distributable>> taskLists = outputMap.get(key);
-    int tasksOfNode = taskLists.size();
-    int i = 0;
-    for (Distributable block : blockOfEachNode) {
-
-      taskLists.get(i % tasksOfNode).add(block);
-      i++;
-    }
-
-  }
-
-  /**
-   * This will create the empty list for each task of a node.
-   *
-   * @param outputMap
-   * @param noOfTasksPerNode
-   * @param key
-   */
-  private static void createTaskListForNode(Map<String, List<List<Distributable>>> outputMap,
-      int noOfTasksPerNode, String key) {
-    List<List<Distributable>> nodeTaskList =
-        new ArrayList<List<Distributable>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (int i = 0; i < noOfTasksPerNode; i++) {
-      List<Distributable> eachTask =
-          new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-      nodeTaskList.add(eachTask);
-
-    }
-    outputMap.put(key, nodeTaskList);
-
-  }
-
-  /**
-   * 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(Map<String, List<Distributable>> outputMap,
-      Set<Distributable> uniqueBlocks, int noOfBlocksPerNode, List<String> activeNodes) {
-
-    if (activeNodes != null) {
-      for (String activeNode : activeNodes) {
-        List<Distributable> blockLst = outputMap.get(activeNode);
-        if (null == blockLst) {
-          blockLst = new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        }
-        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
-        if (blockLst.size() > 0) {
-          outputMap.put(activeNode, blockLst);
-        }
-      }
-    } else {
-      for (Map.Entry<String, List<Distributable>> entry : outputMap.entrySet()) {
-        List<Distributable> blockLst = entry.getValue();
-        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
-      }
-
-    }
-
-    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();
-      }
-    }
-  }
-
-  /**
-   * The method populate the blockLst to be allocate to a specific node.
-   * @param uniqueBlocks
-   * @param noOfBlocksPerNode
-   * @param blockLst
-   */
-  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();
-      blockLst.add(block);
-      blocks.remove();
-      if (blockLst.size() >= noOfBlocksPerNode) {
-        break;
-      }
-    }
-  }
-
-  /**
-   * To create the final output of the Node and Data blocks
-   *
-   * @param outputMap
-   * @param blocksPerNode
-   * @param uniqueBlocks
-   * @param nodeAndBlockMapping
-   * @param activeNodes
-   */
-  private static void createOutputMap(Map<String, List<Distributable>> outputMap, int blocksPerNode,
-      Set<Distributable> uniqueBlocks, Map<String, List<Distributable>> nodeAndBlockMapping,
-      List<String> activeNodes) {
-
-    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);
-
-    for (NodeMultiBlockRelation nodeMultiBlockRelation : multiBlockRelations) {
-      String nodeName = nodeMultiBlockRelation.getNode();
-      //assign the block to the node only if the node is active
-      String activeExecutor = nodeName;
-      if (null != activeNodes) {
-        activeExecutor = getActiveExecutor(activeNodes, nodeName);
-        if (null == activeExecutor) {
-          continue;
-        }
-      }
-      // this loop will be for each NODE
-      int nodeCapacity = 0;
-      // loop thru blocks of each Node
-      for (Distributable block : nodeMultiBlockRelation.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++;
-            uniqueBlocks.remove(block);
-          } else {
-            // No need to continue loop as node is full
-            break;
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * method validates whether the node is active or not.
-   *
-   * @param activeNode
-   * @param nodeName
-   * @return returns true if active else false.
-   */
-  private static String getActiveExecutor(List activeNode, String nodeName) {
-    boolean isActiveNode = activeNode.contains(nodeName);
-    if (isActiveNode) {
-      return nodeName;
-    }
-    //if localhost then retrieve the localhost name then do the check
-    else if (nodeName.equals("localhost")) {
-      try {
-        String hostName = InetAddress.getLocalHost().getHostName();
-        isActiveNode = activeNode.contains(hostName);
-        if (isActiveNode) {
-          return hostName;
-        }
-      } catch (UnknownHostException ue) {
-        isActiveNode = false;
-      }
-    } else {
-      try {
-        String hostAddress = InetAddress.getByName(nodeName).getHostAddress();
-        isActiveNode = activeNode.contains(hostAddress);
-        if (isActiveNode) {
-          return hostAddress;
-        }
-      } catch (UnknownHostException ue) {
-        isActiveNode = false;
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Create the Node and its related blocks Mapping and put in a Map
-   *
-   * @param flattenedList
-   * @param nodeAndBlockMapping
-   */
-  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 (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);
-      }
-    }
-  }
-
-  /**
-   * This method will get the store location for the given path, segment id and partition id
-   *
-   * @param carbonStorePath
-   * @param segmentId
-   */
-  public static void checkAndCreateCarbonDataLocation(String carbonStorePath,
-      String segmentId, CarbonTable carbonTable) {
-    CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
-    String carbonDataDirectoryPath =
-        carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
-    CarbonUtil.checkAndCreateFolder(carbonDataDirectoryPath);
-  }
-
-  /**
-   * return the Array of available local-dirs
-   *
-   * @param conf
-   * @return
-   */
-  public static String[] getConfiguredLocalDirs(SparkConf conf) {
-    return Utils.getConfiguredLocalDirs(conf);
-  }
-
-  /**
-   * 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;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFolders.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFolders.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFolders.java
deleted file mode 100644
index 59ac2f6..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/DeleteLoadFolders.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.spark.load;
-
-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.filesystem.CarbonFileFilter;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
-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 dbName
-   * @param tableName
-   * @param storeLocation
-   * @param partitionId
-   * @param oneLoad
-   * @return
-   */
-  private static String getSegmentPath(String dbName, String tableName, String storeLocation,
-      int partitionId, LoadMetadataDetails oneLoad) {
-    CarbonTablePath carbon = new CarbonStorePath(storeLocation).getCarbonTablePath(
-        new CarbonTableIdentifier(dbName, tableName, ""));
-    String segmentId = oneLoad.getLoadName();
-    return carbon.getCarbonDataDirectoryPath("" + partitionId, segmentId);
-  }
-
-  private static boolean physicalFactAndMeasureMetadataDeletion(String path) {
-
-    boolean status = false;
-    try {
-      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());
-            status = false;
-          }
-        }
-
-      } else {
-        status = false;
-      }
-    } catch (IOException e) {
-      LOGGER.warn("Unable to delete the file as per delete command " + path);
-    }
-
-    return status;
-
-  }
-
-  private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((CarbonCommonConstants.MARKED_FOR_DELETE.equalsIgnoreCase(oneLoad.getLoadStatus())
-        || CarbonCommonConstants.COMPACTED.equalsIgnoreCase(oneLoad.getLoadStatus()))
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrdeletionTimesStamp();
-
-      return CarbonUpdateUtil.isMaxQueryTimeoutExceeded(deletionTime);
-
-    }
-
-    return false;
-  }
-
-  public static boolean deleteLoadFoldersFromFileSystem(String dbName, String tableName,
-      String storeLocation, boolean isForceDelete, LoadMetadataDetails[] details) {
-
-    boolean isDeleted = false;
-
-    if (details != null && details.length != 0) {
-      for (LoadMetadataDetails oneLoad : details) {
-        if (checkIfLoadCanBeDeleted(oneLoad, isForceDelete)) {
-          String path = getSegmentPath(dbName, tableName, storeLocation, 0, oneLoad);
-          boolean deletionStatus = physicalFactAndMeasureMetadataDeletion(path);
-          if (deletionStatus) {
-            isDeleted = true;
-            oneLoad.setVisibility("false");
-            LOGGER.info("Info: Deleted the load " + oneLoad.getLoadName());
-          }
-        }
-      }
-    }
-
-    return isDeleted;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/FailureCauses.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/FailureCauses.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/FailureCauses.java
deleted file mode 100644
index 0345fcd..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/load/FailureCauses.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.spark.load;
-
-/**
- * This Enum is used to determine the Reasons of Failure.
- */
-public enum FailureCauses {
-  NONE,
-  BAD_RECORDS,
-  EXECUTOR_FAILURE,
-  STATUS_FILE_UPDATION_FAILURE,
-  MULTIPLE_INPUT_ROWS_MATCHING
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.java
deleted file mode 100644
index 112fa63..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.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.spark.partition.api;
-
-import java.util.List;
-
-public interface DataPartitioner {
-
-  /**
-   * All the partitions built by the Partitioner
-   */
-  List<Partition> getAllPartitions();
-
-  /**
-   * Identifies the partitions applicable for the given filter (API used for For query)
-   */
-  List<Partition> getPartitions();
-
-}
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/Partition.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/Partition.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/Partition.java
deleted file mode 100644
index fd93ce3..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/Partition.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.spark.partition.api;
-
-import java.io.Serializable;
-import java.util.List;
-
-public interface Partition extends Serializable {
-  /**
-   * unique identification for the partition in the cluster.
-   */
-  String getUniqueID();
-
-  /**
-   * result
-   *
-   * @return
-   */
-  List<String> getFilesPath();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java
deleted file mode 100644
index fa7c4d5..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.spark.partition.api.impl;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.spark.partition.api.Partition;
-
-/**
- * A sample load balancer to distribute the partitions to the available nodes in a round robin mode.
- */
-public class DefaultLoadBalancer {
-  private Map<String, List<Partition>> nodeToPartitonMap =
-      new HashMap<String, List<Partition>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  private Map<Partition, String> partitonToNodeMap =
-      new HashMap<Partition, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  public DefaultLoadBalancer(List<String> nodes, List<Partition> partitions) {
-    //Per form a round robin allocation
-    int nodeCount = nodes.size();
-
-    int partitioner = 0;
-    for (Partition partition : partitions) {
-      int nodeindex = partitioner % nodeCount;
-      String node = nodes.get(nodeindex);
-
-      List<Partition> oldList = nodeToPartitonMap.get(node);
-      if (oldList == null) {
-        oldList = new ArrayList<Partition>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-        nodeToPartitonMap.put(node, oldList);
-      }
-      oldList.add(partition);
-
-      partitonToNodeMap.put(partition, node);
-
-      partitioner++;
-    }
-  }
-
-  public String getNodeForPartitions(Partition partition) {
-    return partitonToNodeMap.get(partition);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionMultiFileImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionMultiFileImpl.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionMultiFileImpl.java
deleted file mode 100644
index c386da1..0000000
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionMultiFileImpl.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.spark.partition.api.impl;
-
-import java.util.List;
-
-import org.apache.carbondata.spark.partition.api.Partition;
-
-public class PartitionMultiFileImpl implements Partition {
-  private static final long serialVersionUID = -4363447826181193976L;
-  private String uniqueID;
-  private List<String> folderPath;
-
-  public PartitionMultiFileImpl(String uniqueID, List<String> folderPath) {
-    this.uniqueID = uniqueID;
-    this.folderPath = folderPath;
-  }
-
-  @Override public String getUniqueID() {
-    // TODO Auto-generated method stub
-    return uniqueID;
-  }
-
-  @Override public List<String> getFilesPath() {
-    // TODO Auto-generated method stub
-    return folderPath;
-  }
-
-}


[48/50] [abbrv] carbondata git commit: [CARBONDATA-1533] Fixed decimal data load fail issue and restricted max characters per column

Posted by ja...@apache.org.
[CARBONDATA-1533] Fixed decimal data load fail issue and restricted max characters per column

1. Fixed data load failure when both precision and data falls in integer range for decimal data type.
2. Restricted max characters per column to 32000 as we use short to store the length of each column

This closes #1395


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

Branch: refs/heads/streaming_ingest
Commit: ad25ffc31ec5d2b74207fc55b2c66bda8443eb7a
Parents: ccdcc3c
Author: manishgupta88 <to...@gmail.com>
Authored: Thu Sep 28 17:03:46 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Oct 10 10:38:17 2017 +0800

----------------------------------------------------------------------
 .../encoding/adaptive/AdaptiveIntegralCodec.java    |  3 +++
 .../src/test/resources/badrecords/bigtab.csv        |  2 +-
 .../src/test/resources/badrecords/bigtabbad.csv     |  3 ---
 .../src/test/resources/decimal_int_range.csv        |  4 ++++
 .../spark/testsuite/bigdecimal/TestBigDecimal.scala | 16 ++++++++++++++++
 .../processing/loading/csvinput/CSVInputFormat.java |  5 ++++-
 6 files changed, 28 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ad25ffc3/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
index c7c10a5..bbc28a6 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/page/encoding/adaptive/AdaptiveIntegralCodec.java
@@ -138,6 +138,9 @@ public class AdaptiveIntegralCodec extends AdaptiveCodec {
         case SHORT_INT:
           encodedPage.putShortInt(rowId, value);
           break;
+        case INT:
+          encodedPage.putInt(rowId, value);
+          break;
         default:
           throw new RuntimeException("internal error: " + debugInfo());
       }


[32/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
deleted file mode 100644
index 50bfaff..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
+++ /dev/null
@@ -1,313 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.datastore.TableSpec;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-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.table.column.CarbonColumn;
-import org.apache.carbondata.processing.newflow.converter.DictionaryCardinalityFinder;
-
-public class CarbonDataLoadConfiguration {
-
-  private DataField[] dataFields;
-
-  private AbsoluteTableIdentifier tableIdentifier;
-
-  private String[] header;
-
-  private String partitionId;
-
-  private String segmentId;
-
-  private String taskNo;
-
-  private BucketingInfo bucketingInfo;
-
-  private Map<String, Object> dataLoadProperties = new HashMap<>();
-
-  /**
-   *  Use one pass to generate dictionary
-   */
-  private boolean useOnePass;
-
-  /**
-   * dictionary server host
-   */
-  private String dictionaryServerHost;
-
-  /**
-   * dictionary sever port
-   */
-  private int dictionaryServerPort;
-
-  private boolean preFetch;
-
-  private int dimensionCount;
-
-  private int measureCount;
-
-  private int noDictionaryCount;
-
-  private int complexColumnCount;
-
-  /**
-   * schema updated time stamp to be used for restructure scenarios
-   */
-  private long schemaUpdatedTimeStamp;
-
-  private DictionaryCardinalityFinder cardinalityFinder;
-
-  private int numberOfSortColumns;
-
-  private int numberOfNoDictSortColumns;
-
-  // contains metadata used in write step of loading process
-  private TableSpec tableSpec;
-
-  public CarbonDataLoadConfiguration() {
-  }
-
-  public void setDataFields(DataField[] dataFields) {
-    this.dataFields = dataFields;
-
-    // set counts for each column category
-    for (DataField dataField : dataFields) {
-      CarbonColumn column = dataField.getColumn();
-      if (column.isDimension()) {
-        dimensionCount++;
-        if (!dataField.hasDictionaryEncoding()) {
-          noDictionaryCount++;
-        }
-      }
-      if (column.isComplex()) {
-        complexColumnCount++;
-      }
-      if (column.isMeasure()) {
-        measureCount++;
-      }
-    }
-  }
-
-  public DataField[] getDataFields() {
-    return dataFields;
-  }
-
-  public int getDimensionCount() {
-    return dimensionCount;
-  }
-
-  public int getNoDictionaryCount() {
-    return noDictionaryCount;
-  }
-
-  public int getComplexColumnCount() {
-    return complexColumnCount;
-  }
-
-  public int getMeasureCount() {
-    return measureCount;
-  }
-
-  public void setNumberOfSortColumns(int numberOfSortColumns) {
-    this.numberOfSortColumns = numberOfSortColumns;
-  }
-
-  public int getNumberOfSortColumns() {
-    return this.numberOfSortColumns;
-  }
-
-  public boolean isSortTable() {
-    return this.numberOfSortColumns > 0;
-  }
-
-  public void setNumberOfNoDictSortColumns(int numberOfNoDictSortColumns) {
-    this.numberOfNoDictSortColumns = numberOfNoDictSortColumns;
-  }
-
-  public int getNumberOfNoDictSortColumns() {
-    return this.numberOfNoDictSortColumns;
-  }
-
-  public String[] getHeader() {
-    return header;
-  }
-
-  public void setHeader(String[] header) {
-    this.header = header;
-  }
-
-  public AbsoluteTableIdentifier getTableIdentifier() {
-    return tableIdentifier;
-  }
-
-  public void setTableIdentifier(AbsoluteTableIdentifier tableIdentifier) {
-    this.tableIdentifier = tableIdentifier;
-  }
-
-  public String getPartitionId() {
-    return partitionId;
-  }
-
-  public void setPartitionId(String partitionId) {
-    this.partitionId = partitionId;
-  }
-
-  public String getSegmentId() {
-    return segmentId;
-  }
-
-  public void setSegmentId(String segmentId) {
-    this.segmentId = segmentId;
-  }
-
-  public String getTaskNo() {
-    return taskNo;
-  }
-
-  public void setTaskNo(String taskNo) {
-    this.taskNo = taskNo;
-  }
-
-  public void setDataLoadProperty(String key, Object value) {
-    dataLoadProperties.put(key, value);
-  }
-
-  public Object getDataLoadProperty(String key) {
-    return dataLoadProperties.get(key);
-  }
-
-  public BucketingInfo getBucketingInfo() {
-    return bucketingInfo;
-  }
-
-  public void setBucketingInfo(BucketingInfo bucketingInfo) {
-    this.bucketingInfo = bucketingInfo;
-  }
-
-  public boolean getUseOnePass() {
-    return useOnePass;
-  }
-
-  public void setUseOnePass(boolean useOnePass) {
-    this.useOnePass = useOnePass;
-  }
-
-  public String getDictionaryServerHost() {
-    return dictionaryServerHost;
-  }
-
-  public void setDictionaryServerHost(String dictionaryServerHost) {
-    this.dictionaryServerHost = dictionaryServerHost;
-  }
-
-  public int getDictionaryServerPort() {
-    return dictionaryServerPort;
-  }
-
-  public void setDictionaryServerPort(int dictionaryServerPort) {
-    this.dictionaryServerPort = dictionaryServerPort;
-  }
-
-  public boolean isPreFetch() {
-    return preFetch;
-  }
-
-  public void setPreFetch(boolean preFetch) {
-    this.preFetch = preFetch;
-  }
-
-  public long getSchemaUpdatedTimeStamp() {
-    return schemaUpdatedTimeStamp;
-  }
-
-  public void setSchemaUpdatedTimeStamp(long schemaUpdatedTimeStamp) {
-    this.schemaUpdatedTimeStamp = schemaUpdatedTimeStamp;
-  }
-
-  public DictionaryCardinalityFinder getCardinalityFinder() {
-    return cardinalityFinder;
-  }
-
-  public void setCardinalityFinder(DictionaryCardinalityFinder cardinalityFinder) {
-    this.cardinalityFinder = cardinalityFinder;
-  }
-
-  public DataType[] getMeasureDataType() {
-    List<Integer> measureIndexes = new ArrayList<>(dataFields.length);
-    int measureCount = 0;
-    for (int i = 0; i < dataFields.length; i++) {
-      if (!dataFields[i].getColumn().isDimension()) {
-        measureIndexes.add(i);
-        measureCount++;
-      }
-    }
-
-    DataType[] type = new DataType[measureCount];
-    for (int i = 0; i < type.length; i++) {
-      type[i] = dataFields[measureIndexes.get(i)].getColumn().getDataType();
-    }
-    return type;
-  }
-
-  public int[] calcDimensionLengths() {
-    int[] dimLensWithComplex = getCardinalityFinder().getCardinality();
-    if (!isSortTable()) {
-      for (int i = 0; i < dimLensWithComplex.length; i++) {
-        if (dimLensWithComplex[i] != 0) {
-          dimLensWithComplex[i] = Integer.MAX_VALUE;
-        }
-      }
-    }
-    List<Integer> dimsLenList = new ArrayList<Integer>();
-    for (int eachDimLen : dimLensWithComplex) {
-      if (eachDimLen != 0) dimsLenList.add(eachDimLen);
-    }
-    int[] dimLens = new int[dimsLenList.size()];
-    for (int i = 0; i < dimsLenList.size(); i++) {
-      dimLens[i] = dimsLenList.get(i);
-    }
-    return dimLens;
-  }
-
-  public KeyGenerator[] createKeyGeneratorForComplexDimension() {
-    int[] dimLens = calcDimensionLengths();
-    KeyGenerator[] complexKeyGenerators = new KeyGenerator[dimLens.length];
-    for (int i = 0; i < dimLens.length; i++) {
-      complexKeyGenerators[i] =
-          KeyGeneratorFactory.getKeyGenerator(new int[] { dimLens[i] });
-    }
-    return complexKeyGenerators;
-  }
-
-  public TableSpec getTableSpec() {
-    return tableSpec;
-  }
-
-  public void setTableSpec(TableSpec tableSpec) {
-    this.tableSpec = tableSpec;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/DataField.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/DataField.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/DataField.java
deleted file mode 100644
index 892055b..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/DataField.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-
-/**
- * Metadata class for each column of table.
- */
-public class DataField implements Serializable {
-
-  public DataField(CarbonColumn column) {
-    this.column = column;
-  }
-
-  private CarbonColumn column;
-
-  private String dateFormat;
-
-  public boolean hasDictionaryEncoding() {
-    return column.hasEncoding(Encoding.DICTIONARY);
-  }
-
-  public CarbonColumn getColumn() {
-    return column;
-  }
-
-  public String getDateFormat() {
-    return dateFormat;
-  }
-
-  public void setDateFormat(String dateFormat) {
-    this.dateFormat = dateFormat;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/DataLoadExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/DataLoadExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/DataLoadExecutor.java
deleted file mode 100644
index 36a89b5..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/DataLoadExecutor.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow;
-
-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.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
-import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.exception.NoRetryException;
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger;
-
-/**
- * It executes the data load.
- */
-public class DataLoadExecutor {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DataLoadExecutor.class.getName());
-
-  private AbstractDataLoadProcessorStep loadProcessorStep;
-
-  private boolean isClosed;
-
-  public void execute(CarbonLoadModel loadModel, String[] storeLocation,
-      CarbonIterator<Object[]>[] inputIterators) throws Exception {
-    try {
-      loadProcessorStep =
-          new DataLoadProcessBuilder().build(loadModel, storeLocation, inputIterators);
-      // 1. initialize
-      loadProcessorStep.initialize();
-      LOGGER.info("Data Loading is started for table " + loadModel.getTableName());
-      // 2. execute the step
-      loadProcessorStep.execute();
-      // check and remove any bad record key from bad record entry logger static map
-      if (badRecordFound(
-          loadModel.getCarbonDataLoadSchema().getCarbonTable().getCarbonTableIdentifier())) {
-        LOGGER.error("Data Load is partially success for table " + loadModel.getTableName());
-      } else {
-        LOGGER.info("Data loading is successful for table " + loadModel.getTableName());
-      }
-    } catch (CarbonDataLoadingException e) {
-      if (e instanceof BadRecordFoundException) {
-        throw new NoRetryException(e.getMessage());
-      } else {
-        throw e;
-      }
-    } catch (Exception e) {
-      LOGGER.error(e, "Data Loading failed for table " + loadModel.getTableName());
-      throw new CarbonDataLoadingException(
-          "Data Loading failed for table " + loadModel.getTableName(), e);
-    } finally {
-      removeBadRecordKey(
-          loadModel.getCarbonDataLoadSchema().getCarbonTable().getCarbonTableIdentifier());
-    }
-  }
-
-  /**
-   * This method will remove any bad record key from the map entry
-   *
-   * @param carbonTableIdentifier
-   * @return
-   */
-  private boolean badRecordFound(CarbonTableIdentifier carbonTableIdentifier) {
-    String badRecordLoggerKey = carbonTableIdentifier.getBadRecordLoggerKey();
-    boolean badRecordKeyFound = false;
-    if (null != BadRecordsLogger.hasBadRecord(badRecordLoggerKey)) {
-      badRecordKeyFound = true;
-    }
-    return badRecordKeyFound;
-  }
-
-  /**
-   * This method will remove the bad record key from bad record logger
-   *
-   * @param carbonTableIdentifier
-   */
-  private void removeBadRecordKey(CarbonTableIdentifier carbonTableIdentifier) {
-    String badRecordLoggerKey = carbonTableIdentifier.getBadRecordLoggerKey();
-    BadRecordsLogger.removeBadRecordKey(badRecordLoggerKey);
-  }
-
-  /**
-   * Method to clean all the resource
-   */
-  public void close() {
-    if (!isClosed && loadProcessorStep != null) {
-      loadProcessorStep.close();
-    }
-    isClosed = true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/DataLoadProcessBuilder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/DataLoadProcessBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/DataLoadProcessBuilder.java
deleted file mode 100644
index ccb25e6..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/DataLoadProcessBuilder.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-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.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.model.CarbonLoadModel;
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
-import org.apache.carbondata.processing.newflow.sort.SortScopeOptions;
-import org.apache.carbondata.processing.newflow.steps.CarbonRowDataWriterProcessorStepImpl;
-import org.apache.carbondata.processing.newflow.steps.DataConverterProcessorStepImpl;
-import org.apache.carbondata.processing.newflow.steps.DataConverterProcessorWithBucketingStepImpl;
-import org.apache.carbondata.processing.newflow.steps.DataWriterBatchProcessorStepImpl;
-import org.apache.carbondata.processing.newflow.steps.DataWriterProcessorStepImpl;
-import org.apache.carbondata.processing.newflow.steps.InputProcessorStepImpl;
-import org.apache.carbondata.processing.newflow.steps.SortProcessorStepImpl;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-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 {
-    CarbonDataLoadConfiguration configuration = createConfiguration(loadModel, storeLocation);
-    SortScopeOptions.SortScope sortScope = CarbonDataProcessorUtil.getSortScope(configuration);
-    if (!configuration.isSortTable() || sortScope.equals(SortScopeOptions.SortScope.NO_SORT)) {
-      return buildInternalForNoSort(inputIterators, configuration);
-    } else if (configuration.getBucketingInfo() != null) {
-      return buildInternalForBucketing(inputIterators, configuration);
-    } else if (sortScope.equals(SortScopeOptions.SortScope.BATCH_SORT)) {
-      return buildInternalForBatchSort(inputIterators, configuration);
-    } else {
-      return buildInternal(inputIterators, configuration);
-    }
-  }
-
-  private AbstractDataLoadProcessorStep buildInternal(CarbonIterator[] inputIterators,
-      CarbonDataLoadConfiguration configuration) {
-    // 1. Reads the data input iterators and parses the data.
-    AbstractDataLoadProcessorStep inputProcessorStep =
-        new InputProcessorStepImpl(configuration, inputIterators);
-    // 2. Converts the data like dictionary or non dictionary or complex objects depends on
-    // data types and configurations.
-    AbstractDataLoadProcessorStep converterProcessorStep =
-        new DataConverterProcessorStepImpl(configuration, inputProcessorStep);
-    // 3. Sorts the data by SortColumn
-    AbstractDataLoadProcessorStep sortProcessorStep =
-        new SortProcessorStepImpl(configuration, converterProcessorStep);
-    // 4. Writes the sorted data in carbondata format.
-    return new DataWriterProcessorStepImpl(configuration, sortProcessorStep);
-  }
-
-  private AbstractDataLoadProcessorStep buildInternalForNoSort(CarbonIterator[] inputIterators,
-      CarbonDataLoadConfiguration configuration) {
-    // 1. Reads the data input iterators and parses the data.
-    AbstractDataLoadProcessorStep inputProcessorStep =
-        new InputProcessorStepImpl(configuration, inputIterators);
-    // 2. Converts the data like dictionary or non dictionary or complex objects depends on
-    // data types and configurations.
-    AbstractDataLoadProcessorStep converterProcessorStep =
-        new DataConverterProcessorStepImpl(configuration, inputProcessorStep);
-    // 3. Writes the sorted data in carbondata format.
-    AbstractDataLoadProcessorStep writerProcessorStep =
-        new CarbonRowDataWriterProcessorStepImpl(configuration, converterProcessorStep);
-    return writerProcessorStep;
-  }
-
-  private AbstractDataLoadProcessorStep buildInternalForBatchSort(CarbonIterator[] inputIterators,
-      CarbonDataLoadConfiguration configuration) {
-    // 1. Reads the data input iterators and parses the data.
-    AbstractDataLoadProcessorStep inputProcessorStep =
-        new InputProcessorStepImpl(configuration, inputIterators);
-    // 2. Converts the data like dictionary or non dictionary or complex objects depends on
-    // data types and configurations.
-    AbstractDataLoadProcessorStep converterProcessorStep =
-        new DataConverterProcessorStepImpl(configuration, inputProcessorStep);
-    // 3. Sorts the data by SortColumn or not
-    AbstractDataLoadProcessorStep sortProcessorStep =
-        new SortProcessorStepImpl(configuration, converterProcessorStep);
-    // 4. Writes the sorted data in carbondata format.
-    return new DataWriterBatchProcessorStepImpl(configuration, sortProcessorStep);
-  }
-
-  private AbstractDataLoadProcessorStep buildInternalForBucketing(CarbonIterator[] inputIterators,
-      CarbonDataLoadConfiguration configuration) throws Exception {
-    // 1. Reads the data input iterators and parses the data.
-    AbstractDataLoadProcessorStep inputProcessorStep =
-        new InputProcessorStepImpl(configuration, inputIterators);
-    // 2. Converts the data like dictionary or non dictionary or complex objects depends on
-    // data types and configurations.
-    AbstractDataLoadProcessorStep converterProcessorStep =
-        new DataConverterProcessorWithBucketingStepImpl(configuration, inputProcessorStep);
-    // 3. Sorts the data by SortColumn or not
-    AbstractDataLoadProcessorStep sortProcessorStep =
-        new SortProcessorStepImpl(configuration, converterProcessorStep);
-    // 4. Writes the sorted data in carbondata format.
-    return new DataWriterProcessorStepImpl(configuration, sortProcessorStep);
-  }
-
-  public static CarbonDataLoadConfiguration createConfiguration(CarbonLoadModel loadModel,
-      String[] storeLocation) {
-    CarbonDataProcessorUtil.createLocations(storeLocation);
-
-    String databaseName = loadModel.getDatabaseName();
-    String tableName = loadModel.getTableName();
-    String tempLocationKey = CarbonDataProcessorUtil
-        .getTempStoreLocationKey(databaseName, tableName, loadModel.getSegmentId(),
-            loadModel.getTaskNo(), false, false);
-    CarbonProperties.getInstance().addProperty(tempLocationKey,
-        StringUtils.join(storeLocation, File.pathSeparator));
-    CarbonProperties.getInstance()
-        .addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS, loadModel.getStorePath());
-
-    return createConfiguration(loadModel);
-  }
-
-  public static CarbonDataLoadConfiguration createConfiguration(CarbonLoadModel loadModel) {
-    CarbonDataLoadConfiguration configuration = new CarbonDataLoadConfiguration();
-    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
-    AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
-    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,
-        new String[] { loadModel.getComplexDelimiterLevel1(),
-            loadModel.getComplexDelimiterLevel2() });
-    configuration.setDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT,
-        loadModel.getSerializationNullFormat().split(",")[1]);
-    configuration.setDataLoadProperty(DataLoadProcessorConstants.FACT_TIME_STAMP,
-        loadModel.getFactTimeStamp());
-    configuration.setDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ENABLE,
-        loadModel.getBadRecordsLoggerEnable().split(",")[1]);
-    configuration.setDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ACTION,
-        loadModel.getBadRecordsAction().split(",")[1]);
-    configuration.setDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD,
-        loadModel.getIsEmptyDataBadRecord().split(",")[1]);
-    configuration.setDataLoadProperty(DataLoadProcessorConstants.FACT_FILE_PATH,
-        loadModel.getFactFilePath());
-    configuration
-        .setDataLoadProperty(CarbonCommonConstants.LOAD_SORT_SCOPE, loadModel.getSortScope());
-    configuration.setDataLoadProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
-        loadModel.getBatchSortSizeInMb());
-    configuration.setDataLoadProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS,
-        loadModel.getGlobalSortPartitions());
-    configuration.setDataLoadProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
-        loadModel.getBadRecordsLocation());
-    CarbonMetadata.getInstance().addCarbonTable(carbonTable);
-    List<CarbonDimension> dimensions =
-        carbonTable.getDimensionByTableName(carbonTable.getFactTableName());
-    List<CarbonMeasure> measures =
-        carbonTable.getMeasureByTableName(carbonTable.getFactTableName());
-    Map<String, String> dateFormatMap =
-        CarbonDataProcessorUtil.getDateFormatMap(loadModel.getDateFormat());
-    List<DataField> dataFields = new ArrayList<>();
-    List<DataField> complexDataFields = new ArrayList<>();
-
-    // First add dictionary and non dictionary dimensions because these are part of mdk key.
-    // And then add complex data types and measures.
-    for (CarbonColumn column : dimensions) {
-      DataField dataField = new DataField(column);
-      dataField.setDateFormat(dateFormatMap.get(column.getColName()));
-      if (column.isComplex()) {
-        complexDataFields.add(dataField);
-      } else {
-        dataFields.add(dataField);
-      }
-    }
-    dataFields.addAll(complexDataFields);
-    for (CarbonColumn column : measures) {
-      // This dummy measure is added when no measure was present. We no need to load it.
-      if (!(column.getColName().equals("default_dummy_measure"))) {
-        dataFields.add(new DataField(column));
-      }
-    }
-    configuration.setDataFields(dataFields.toArray(new DataField[dataFields.size()]));
-    configuration.setBucketingInfo(carbonTable.getBucketingInfo(carbonTable.getFactTableName()));
-    // configuration for one pass load: dictionary server info
-    configuration.setUseOnePass(loadModel.getUseOnePass());
-    configuration.setDictionaryServerHost(loadModel.getDictionaryServerHost());
-    configuration.setDictionaryServerPort(loadModel.getDictionaryServerPort());
-    configuration.setPreFetch(loadModel.isPreFetch());
-    configuration.setNumberOfSortColumns(carbonTable.getNumberOfSortColumns());
-    configuration.setNumberOfNoDictSortColumns(carbonTable.getNumberOfNoDictSortColumns());
-
-    TableSpec tableSpec = new TableSpec(dimensions, measures);
-    configuration.setTableSpec(tableSpec);
-    return configuration;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/complexobjects/ArrayObject.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/complexobjects/ArrayObject.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/complexobjects/ArrayObject.java
deleted file mode 100644
index 196afdb..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/complexobjects/ArrayObject.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.processing.newflow.complexobjects;
-
-public class ArrayObject {
-
-  private Object[] data;
-
-  public ArrayObject(Object[] data) {
-    this.data = data;
-  }
-
-  public Object[] getData() {
-    return data;
-  }
-
-  public void setData(Object[] data) {
-    this.data = data;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/complexobjects/StructObject.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/complexobjects/StructObject.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/complexobjects/StructObject.java
deleted file mode 100644
index d1e0f9b..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/complexobjects/StructObject.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.complexobjects;
-
-public class StructObject {
-
-  private Object[] data;
-
-  public StructObject(Object[] data) {
-    this.data = data;
-  }
-
-  public Object[] getData() {
-    return data;
-  }
-
-  public void setData(Object[] data) {
-    this.data = data;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/constants/DataLoadProcessorConstants.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/constants/DataLoadProcessorConstants.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/constants/DataLoadProcessorConstants.java
deleted file mode 100644
index 11570b4..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/constants/DataLoadProcessorConstants.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.constants;
-
-/**
- * Constants used in data loading.
- */
-public final class DataLoadProcessorConstants {
-
-  public static final String FACT_TIME_STAMP = "FACT_TIME_STAMP";
-
-  public static final String COMPLEX_DELIMITERS = "COMPLEX_DELIMITERS";
-
-  public static final String SERIALIZATION_NULL_FORMAT = "SERIALIZATION_NULL_FORMAT";
-
-  public static final String BAD_RECORDS_LOGGER_ENABLE = "BAD_RECORDS_LOGGER_ENABLE";
-
-  public static final String BAD_RECORDS_LOGGER_ACTION = "BAD_RECORDS_LOGGER_ACTION";
-
-  public static final String IS_EMPTY_DATA_BAD_RECORD = "IS_EMPTY_DATA_BAD_RECORD";
-
-  public static final String FACT_FILE_PATH = "FACT_FILE_PATH";
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/BadRecordLogHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/BadRecordLogHolder.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/BadRecordLogHolder.java
deleted file mode 100644
index f7ce620..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/BadRecordLogHolder.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.converter;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * It is holder for reason of bad records.
- */
-public class BadRecordLogHolder {
-
-  /**
-   * this map will hold the bad record unified message for columns
-   */
-  private Map<String, String> columnMessageMap = new HashMap<>();
-
-  private String reason;
-
-  private boolean badRecordAdded;
-
-  private boolean isLogged;
-
-  public String getReason() {
-    return reason;
-  }
-
-  public void setReason(String reason) {
-    this.reason = reason;
-    badRecordAdded = true;
-  }
-
-  public boolean isBadRecordNotAdded() {
-    return badRecordAdded;
-  }
-
-  public void clear() {
-    this.badRecordAdded = false;
-  }
-
-  public boolean isLogged() {
-    return isLogged;
-  }
-
-  public void setLogged(boolean logged) {
-    isLogged = logged;
-  }
-
-  public Map<String, String> getColumnMessageMap() {
-    return columnMessageMap;
-  }
-
-  /**
-   * this method will clear the map entries
-   */
-  public void finish() {
-    if (null != columnMessageMap) {
-      columnMessageMap.clear();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/DictionaryCardinalityFinder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/DictionaryCardinalityFinder.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/DictionaryCardinalityFinder.java
deleted file mode 100644
index 751f909..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/DictionaryCardinalityFinder.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.processing.newflow.converter;
-
-/**
- * Finds the current cardinality of dimensions.
- */
-public interface DictionaryCardinalityFinder {
-
-  int[] getCardinality();
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/FieldConverter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/FieldConverter.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/FieldConverter.java
deleted file mode 100644
index 88828be..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/FieldConverter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.converter;
-
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-
-/**
- * This interface converts/transforms the column field.
- */
-public interface FieldConverter {
-
-  /**
-   * It converts the column field and updates the data in same location/index in row.
-   * @param row
-   * @return the status whether it could be loaded or not, usually when record is added
-   * to bad records then it returns false.
-   * @throws CarbonDataLoadingException
-   */
-  void convert(CarbonRow row, BadRecordLogHolder logHolder) throws CarbonDataLoadingException;
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java
deleted file mode 100644
index f4876db..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/RowConverter.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.converter;
-
-import java.io.IOException;
-
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-
-/**
- * convert the row
- */
-public interface RowConverter extends DictionaryCardinalityFinder {
-
-  void initialize() throws IOException;
-
-  CarbonRow convert(CarbonRow row) throws CarbonDataLoadingException;
-
-  RowConverter createCopyForNewThread();
-
-  void finish();
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/AbstractDictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/AbstractDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/AbstractDictionaryFieldConverterImpl.java
deleted file mode 100644
index bbbf5e6..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/AbstractDictionaryFieldConverterImpl.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.converter.impl;
-
-import java.util.List;
-
-import org.apache.carbondata.processing.newflow.converter.FieldConverter;
-
-public abstract class AbstractDictionaryFieldConverterImpl implements FieldConverter {
-
-  public abstract void fillColumnCardinality(List<Integer> cardinality);
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/ComplexFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/ComplexFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/ComplexFieldConverterImpl.java
deleted file mode 100644
index 8feea6a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/ComplexFieldConverterImpl.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.converter.impl;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.util.List;
-
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.newflow.converter.BadRecordLogHolder;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-
-public class ComplexFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
-
-  private GenericDataType genericDataType;
-
-  private int index;
-
-  public ComplexFieldConverterImpl(GenericDataType genericDataType, int index) {
-    this.genericDataType = genericDataType;
-    this.index = index;
-  }
-
-  @Override
-  public void convert(CarbonRow row, BadRecordLogHolder logHolder) {
-    Object object = row.getObject(index);
-    // TODO Its temporary, needs refactor here.
-    ByteArrayOutputStream byteArray = new ByteArrayOutputStream();
-    DataOutputStream dataOutputStream = new DataOutputStream(byteArray);
-    try {
-      genericDataType.writeByteArray(object, dataOutputStream);
-      dataOutputStream.close();
-      row.update(byteArray.toByteArray(), index);
-    } catch (Exception e) {
-      throw new CarbonDataLoadingException(object + "", e);
-    }
-  }
-
-  @Override public void fillColumnCardinality(List<Integer> cardinality) {
-    genericDataType.fillCardinality(cardinality);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
deleted file mode 100644
index 8d4d5a3..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DictionaryFieldConverterImpl.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.converter.impl;
-
-import java.io.IOException;
-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.cache.Cache;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-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.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.converter.BadRecordLogHolder;
-import org.apache.carbondata.processing.newflow.dictionary.DictionaryServerClientDictionary;
-import org.apache.carbondata.processing.newflow.dictionary.PreCreatedDictionary;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DictionaryFieldConverterImpl.class.getName());
-
-  private BiDictionary<Integer, Object> dictionaryGenerator;
-
-  private int index;
-
-  private CarbonDimension carbonDimension;
-
-  private String nullFormat;
-
-  private Dictionary dictionary;
-
-  private DictionaryMessage dictionaryMessage;
-
-  private boolean isEmptyBadRecord;
-
-  public DictionaryFieldConverterImpl(DataField dataField,
-      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, String nullFormat, int index,
-      DictionaryClient client, boolean useOnePass, String storePath,
-      Map<Object, Integer> localCache, boolean isEmptyBadRecord) throws IOException {
-    this.index = index;
-    this.carbonDimension = (CarbonDimension) dataField.getColumn();
-    this.nullFormat = nullFormat;
-    this.isEmptyBadRecord = isEmptyBadRecord;
-    DictionaryColumnUniqueIdentifier identifier =
-        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
-            dataField.getColumn().getColumnIdentifier(), dataField.getColumn().getDataType(),
-            CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
-
-    // if use one pass, use DictionaryServerClientDictionary
-    if (useOnePass) {
-      if (CarbonUtil.isFileExistsForGivenColumn(storePath, identifier)) {
-        dictionary = cache.get(identifier);
-      }
-      dictionaryMessage = new DictionaryMessage();
-      dictionaryMessage.setColumnName(dataField.getColumn().getColName());
-      // for table initialization
-      dictionaryMessage.setTableUniqueId(carbonTableIdentifier.getTableId());
-      dictionaryMessage.setData("0");
-      // for generate dictionary
-      dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);
-      dictionaryGenerator = new DictionaryServerClientDictionary(dictionary, client,
-          dictionaryMessage, localCache);
-    } else {
-      dictionary = cache.get(identifier);
-      dictionaryGenerator = new PreCreatedDictionary(dictionary);
-    }
-  }
-
-  @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder)
-      throws CarbonDataLoadingException {
-    try {
-      String parsedValue = null;
-      String dimensionValue = row.getString(index);
-      if (dimensionValue == null || dimensionValue.equals(nullFormat)) {
-        parsedValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
-      } else {
-        parsedValue = DataTypeUtil.parseValue(dimensionValue, carbonDimension);
-      }
-      if (null == parsedValue) {
-        if ((dimensionValue.length() > 0) || (dimensionValue.length() == 0 && isEmptyBadRecord)) {
-          String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
-          if (null == message) {
-            message = CarbonDataProcessorUtil.prepareFailureReason(
-                carbonDimension.getColName(), carbonDimension.getDataType());
-            logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
-          } logHolder.setReason(message);
-        }
-        row.update(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY, index);
-      } else {
-        row.update(dictionaryGenerator.getOrGenerateKey(parsedValue), index);
-      }
-    } catch (DictionaryGenerationException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-  }
-
-  @Override
-  public void fillColumnCardinality(List<Integer> cardinality) {
-    cardinality.add(dictionaryGenerator.size());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DirectDictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DirectDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DirectDictionaryFieldConverterImpl.java
deleted file mode 100644
index f269274..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/DirectDictionaryFieldConverterImpl.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.converter.impl;
-
-import java.util.List;
-
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.converter.BadRecordLogHolder;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class DirectDictionaryFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
-
-  private DirectDictionaryGenerator directDictionaryGenerator;
-
-  private int index;
-
-  private String nullFormat;
-
-  private CarbonColumn column;
-  private boolean isEmptyBadRecord;
-
-  public DirectDictionaryFieldConverterImpl(DataField dataField, String nullFormat, int index,
-      boolean isEmptyBadRecord) {
-    this.nullFormat = nullFormat;
-    this.column = dataField.getColumn();
-    if (dataField.getDateFormat() != null && !dataField.getDateFormat().isEmpty()) {
-      this.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-          .getDirectDictionaryGenerator(dataField.getColumn().getDataType(),
-              dataField.getDateFormat());
-
-    } else {
-      this.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-          .getDirectDictionaryGenerator(dataField.getColumn().getDataType());
-    }
-    this.index = index;
-    this.isEmptyBadRecord = isEmptyBadRecord;
-  }
-
-  @Override
-  public void convert(CarbonRow row, BadRecordLogHolder logHolder) {
-    String value = row.getString(index);
-    if (value == null) {
-      logHolder.setReason(
-          CarbonDataProcessorUtil.prepareFailureReason(column.getColName(), column.getDataType()));
-      row.update(1, index);
-    } else if (value.equals(nullFormat)) {
-      row.update(1, index);
-    } else {
-      int key = directDictionaryGenerator.generateDirectSurrogateKey(value);
-      if (key == 1) {
-        if ((value.length() > 0) || (value.length() == 0 && isEmptyBadRecord)) {
-          String message = logHolder.getColumnMessageMap().get(column.getColName());
-          if (null == message) {
-            message = CarbonDataProcessorUtil.prepareFailureReason(
-                column.getColName(), column.getDataType());
-            logHolder.getColumnMessageMap().put(column.getColName(), message);
-          }
-          logHolder.setReason(message);
-        }
-      }
-      row.update(key, index);
-    }
-  }
-
-  @Override
-  public void fillColumnCardinality(List<Integer> cardinality) {
-    cardinality.add(Integer.MAX_VALUE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
deleted file mode 100644
index 1aada19..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/FieldEncoderFactory.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.converter.impl;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.processing.datatypes.ArrayDataType;
-import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.datatypes.PrimitiveDataType;
-import org.apache.carbondata.processing.datatypes.StructDataType;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.converter.FieldConverter;
-
-public class FieldEncoderFactory {
-
-  private static FieldEncoderFactory instance;
-
-  private FieldEncoderFactory() {
-
-  }
-
-  public static FieldEncoderFactory getInstance() {
-    if (instance == null) {
-      instance = new FieldEncoderFactory();
-    }
-    return instance;
-  }
-
-  /**
-   * Creates the FieldConverter for all dimensions, for measures return null.
-   *
-   * @param dataField             column schema
-   * @param cache                 dicionary cache.
-   * @param carbonTableIdentifier table identifier
-   * @param index                 index of column in the row.
-   * @param isEmptyBadRecord
-   * @return
-   */
-  public FieldConverter createFieldEncoder(DataField dataField,
-      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, int index, String nullFormat,
-      DictionaryClient client, Boolean useOnePass, String storePath,
-      Map<Object, Integer> localCache, boolean isEmptyBadRecord)
-      throws IOException {
-    // Converters are only needed for dimensions and measures it return null.
-    if (dataField.getColumn().isDimension()) {
-      if (dataField.getColumn().hasEncoding(Encoding.DIRECT_DICTIONARY) &&
-          !dataField.getColumn().isComplex()) {
-        return new DirectDictionaryFieldConverterImpl(dataField, nullFormat, index,
-            isEmptyBadRecord);
-      } else if (dataField.getColumn().hasEncoding(Encoding.DICTIONARY) &&
-          !dataField.getColumn().isComplex()) {
-        return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier, nullFormat,
-            index, client, useOnePass, storePath, localCache, isEmptyBadRecord);
-      } else if (dataField.getColumn().isComplex()) {
-        return new ComplexFieldConverterImpl(
-            createComplexType(dataField, cache, carbonTableIdentifier,
-                    client, useOnePass, storePath, localCache), index);
-      } else {
-        return new NonDictionaryFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord);
-      }
-    } else {
-      return new MeasureFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord);
-    }
-  }
-
-  /**
-   * Create parser for the carbon column.
-   */
-  private static GenericDataType createComplexType(DataField dataField,
-      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      String storePath, Map<Object, Integer> localCache) {
-    return createComplexType(dataField.getColumn(), dataField.getColumn().getColName(), cache,
-        carbonTableIdentifier, client, useOnePass, storePath, localCache);
-  }
-
-  /**
-   * This method may be called recursively if the carbon column is complex type.
-   *
-   * @return GenericDataType
-   */
-  private static GenericDataType createComplexType(CarbonColumn carbonColumn, String parentName,
-      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
-      CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
-      String storePath, Map<Object, Integer> localCache) {
-    switch (carbonColumn.getDataType()) {
-      case ARRAY:
-        List<CarbonDimension> listOfChildDimensions =
-            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
-        // Create array parser with complex delimiter
-        ArrayDataType arrayDataType =
-            new ArrayDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
-        for (CarbonDimension dimension : listOfChildDimensions) {
-          arrayDataType.addChildren(createComplexType(dimension, carbonColumn.getColName(), cache,
-              carbonTableIdentifier, client, useOnePass, storePath, localCache));
-        }
-        return arrayDataType;
-      case STRUCT:
-        List<CarbonDimension> dimensions =
-            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
-        // Create struct parser with complex delimiter
-        StructDataType structDataType =
-            new StructDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
-        for (CarbonDimension dimension : dimensions) {
-          structDataType.addChildren(createComplexType(dimension, carbonColumn.getColName(), cache,
-              carbonTableIdentifier, client, useOnePass, storePath, localCache));
-        }
-        return structDataType;
-      case MAP:
-        throw new UnsupportedOperationException("Complex type Map is not supported yet");
-      default:
-        return new PrimitiveDataType(carbonColumn.getColName(), parentName,
-            carbonColumn.getColumnId(), (CarbonDimension) carbonColumn, cache,
-            carbonTableIdentifier, client, useOnePass, storePath, localCache);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/MeasureFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/MeasureFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/MeasureFieldConverterImpl.java
deleted file mode 100644
index 8e20b8f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/MeasureFieldConverterImpl.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.converter.impl;
-
-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.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.converter.BadRecordLogHolder;
-import org.apache.carbondata.processing.newflow.converter.FieldConverter;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-/**
- * Converter for measure
- */
-public class MeasureFieldConverterImpl implements FieldConverter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(MeasureFieldConverterImpl.class.getName());
-
-  private int index;
-
-  private DataType dataType;
-
-  private CarbonMeasure measure;
-
-  private String nullformat;
-
-  private boolean isEmptyBadRecord;
-
-  public MeasureFieldConverterImpl(DataField dataField, String nullformat, int index,
-      boolean isEmptyBadRecord) {
-    this.dataType = dataField.getColumn().getDataType();
-    this.measure = (CarbonMeasure) dataField.getColumn();
-    this.nullformat = nullformat;
-    this.index = index;
-    this.isEmptyBadRecord = isEmptyBadRecord;
-  }
-
-  @Override
-  public void convert(CarbonRow row, BadRecordLogHolder logHolder)
-      throws CarbonDataLoadingException {
-    String value = row.getString(index);
-    Object output;
-    boolean isNull = CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(value);
-    if (value == null || isNull) {
-      String message = logHolder.getColumnMessageMap().get(measure.getColName());
-      if (null == message) {
-        message = CarbonDataProcessorUtil
-            .prepareFailureReason(measure.getColName(), measure.getDataType());
-        logHolder.getColumnMessageMap().put(measure.getColName(), message);
-      }
-      row.update(null, index);
-    } else if (value.length() == 0) {
-      if (isEmptyBadRecord) {
-        String message = logHolder.getColumnMessageMap().get(measure.getColName());
-        if (null == message) {
-          message = CarbonDataProcessorUtil
-              .prepareFailureReason(measure.getColName(), measure.getDataType());
-          logHolder.getColumnMessageMap().put(measure.getColName(), message);
-        }
-        logHolder.setReason(message);
-      }
-      row.update(null, index);
-    } else if (value.equals(nullformat)) {
-      row.update(null, index);
-    } else {
-      try {
-        output = DataTypeUtil.getMeasureValueBasedOnDataType(value, dataType, measure);
-        row.update(output, index);
-      } catch (NumberFormatException e) {
-        LOGGER.warn(
-            "Cant not convert value to Numeric type value. Value considered as null.");
-        logHolder.setReason(
-            CarbonDataProcessorUtil.prepareFailureReason(measure.getColName(), dataType));
-        output = null;
-        row.update(output, index);
-      }
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/NonDictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/NonDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/NonDictionaryFieldConverterImpl.java
deleted file mode 100644
index 4861d78..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/NonDictionaryFieldConverterImpl.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.converter.impl;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.converter.BadRecordLogHolder;
-import org.apache.carbondata.processing.newflow.converter.FieldConverter;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class NonDictionaryFieldConverterImpl implements FieldConverter {
-
-  private DataType dataType;
-
-  private int index;
-
-  private String nullformat;
-
-  private CarbonColumn column;
-
-  private boolean isEmptyBadRecord;
-
-  private DataField dataField;
-
-  public NonDictionaryFieldConverterImpl(DataField dataField, String nullformat, int index,
-      boolean isEmptyBadRecord) {
-    this.dataField = dataField;
-    this.dataType = dataField.getColumn().getDataType();
-    this.column = dataField.getColumn();
-    this.index = index;
-    this.nullformat = nullformat;
-    this.isEmptyBadRecord = isEmptyBadRecord;
-  }
-
-  @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder) {
-    String dimensionValue = row.getString(index);
-    if (null == dimensionValue && column.getDataType() != DataType.STRING) {
-      logHolder.setReason(
-          CarbonDataProcessorUtil.prepareFailureReason(column.getColName(), column.getDataType()));
-      updateWithNullValue(row);
-    } else if (dimensionValue == null || dimensionValue.equals(nullformat)) {
-      updateWithNullValue(row);
-    } else {
-      try {
-        row.update(DataTypeUtil
-            .getBytesBasedOnDataTypeForNoDictionaryColumn(dimensionValue, dataType,
-                dataField.getDateFormat()), index);
-      } catch (Throwable ex) {
-        if (dimensionValue.length() > 0 || (dimensionValue.length() == 0 && isEmptyBadRecord)) {
-          String message = logHolder.getColumnMessageMap().get(column.getColName());
-          if (null == message) {
-            message = CarbonDataProcessorUtil
-                .prepareFailureReason(column.getColName(), column.getDataType());
-            logHolder.getColumnMessageMap().put(column.getColName(), message);
-          }
-          logHolder.setReason(message);
-          updateWithNullValue(row);
-        } else {
-          updateWithNullValue(row);
-        }
-      }
-    }
-  }
-
-  private void updateWithNullValue(CarbonRow row) {
-    if (dataType == DataType.STRING) {
-      row.update(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, index);
-    } else {
-      row.update(CarbonCommonConstants.EMPTY_BYTE_ARRAY, index);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.java
deleted file mode 100644
index eecb0e9..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/converter/impl/RowConverterImpl.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.processing.newflow.converter.impl;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.cache.Cache;
-import org.apache.carbondata.core.cache.CacheProvider;
-import org.apache.carbondata.core.cache.CacheType;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
-import org.apache.carbondata.processing.newflow.converter.BadRecordLogHolder;
-import org.apache.carbondata.processing.newflow.converter.FieldConverter;
-import org.apache.carbondata.processing.newflow.converter.RowConverter;
-import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger;
-
-/**
- * It converts the complete row if necessary, dictionary columns are encoded with dictionary values
- * and nondictionary values are converted to binary.
- */
-public class RowConverterImpl implements RowConverter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(RowConverterImpl.class.getName());
-
-  private CarbonDataLoadConfiguration configuration;
-
-  private DataField[] fields;
-
-  private FieldConverter[] fieldConverters;
-
-  private BadRecordsLogger badRecordLogger;
-
-  private BadRecordLogHolder logHolder;
-
-  private List<DictionaryClient> dictClients = new ArrayList<>();
-
-  private ExecutorService executorService;
-
-  private Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache;
-
-  private Map<Object, Integer>[] localCaches;
-
-  public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration,
-      BadRecordsLogger badRecordLogger) {
-    this.fields = fields;
-    this.configuration = configuration;
-    this.badRecordLogger = badRecordLogger;
-  }
-
-  @Override
-  public void initialize() throws IOException {
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    cache = cacheProvider.createCache(CacheType.REVERSE_DICTIONARY,
-        configuration.getTableIdentifier().getStorePath());
-    String nullFormat =
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
-            .toString();
-    boolean isEmptyBadRecord = Boolean.parseBoolean(
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD)
-            .toString());
-    List<FieldConverter> fieldConverterList = new ArrayList<>();
-    localCaches = new Map[fields.length];
-    long lruCacheStartTime = System.currentTimeMillis();
-    DictionaryClient client = createDictionaryClient();
-    dictClients.add(client);
-
-    for (int i = 0; i < fields.length; i++) {
-      localCaches[i] = new ConcurrentHashMap<>();
-      FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
-          .createFieldEncoder(fields[i], cache,
-              configuration.getTableIdentifier().getCarbonTableIdentifier(), i, nullFormat, client,
-              configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(),
-              localCaches[i], isEmptyBadRecord);
-      fieldConverterList.add(fieldConverter);
-    }
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-        .recordLruCacheLoadTime((System.currentTimeMillis() - lruCacheStartTime) / 1000.0);
-    fieldConverters = fieldConverterList.toArray(new FieldConverter[fieldConverterList.size()]);
-    logHolder = new BadRecordLogHolder();
-  }
-
-  private DictionaryClient createDictionaryClient() {
-    // for one pass load, start the dictionary client
-    if (configuration.getUseOnePass()) {
-      if (executorService == null) {
-        executorService = Executors.newCachedThreadPool();
-      }
-      Future<DictionaryClient> result = executorService.submit(new Callable<DictionaryClient>() {
-        @Override
-        public DictionaryClient call() throws Exception {
-          Thread.currentThread().setName("Dictionary client");
-          DictionaryClient dictionaryClient = new DictionaryClient();
-          dictionaryClient.startClient(configuration.getDictionaryServerHost(),
-              configuration.getDictionaryServerPort());
-          return dictionaryClient;
-        }
-      });
-
-      try {
-        // wait for client initialization finished, or will raise null pointer exception
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        LOGGER.error(e);
-        throw new RuntimeException(e);
-      }
-
-      try {
-        return result.get();
-      } catch (InterruptedException | ExecutionException e) {
-        throw new RuntimeException(e);
-      }
-    }
-    return null;
-  }
-
-  @Override
-  public CarbonRow convert(CarbonRow row) throws CarbonDataLoadingException {
-    //TODO: only copy if it is bad record
-    CarbonRow copy = row.getCopy();
-    logHolder.setLogged(false);
-    logHolder.clear();
-    for (int i = 0; i < fieldConverters.length; i++) {
-      fieldConverters[i].convert(row, logHolder);
-      if (!logHolder.isLogged() && logHolder.isBadRecordNotAdded()) {
-        badRecordLogger.addBadRecordsToBuilder(copy.getData(), logHolder.getReason());
-        if (badRecordLogger.isDataLoadFail()) {
-          String error = "Data load failed due to bad record: " + logHolder.getReason() +
-              "Please enable bad record logger to know the detail reason.";
-          throw new BadRecordFoundException(error);
-        }
-        logHolder.clear();
-        logHolder.setLogged(true);
-        if (badRecordLogger.isBadRecordConvertNullDisable()) {
-          return null;
-        }
-      }
-    }
-    return row;
-  }
-
-  @Override
-  public void finish() {
-    // close dictionary client when finish write
-    if (configuration.getUseOnePass()) {
-      for (DictionaryClient client : dictClients) {
-        if (client != null) {
-          client.shutDown();
-        }
-      }
-      if (null != logHolder) {
-        logHolder.finish();
-      }
-      if (executorService != null) {
-        executorService.shutdownNow();
-        executorService = null;
-      }
-    }
-  }
-
-  @Override
-  public RowConverter createCopyForNewThread() {
-    RowConverterImpl converter =
-        new RowConverterImpl(this.fields, this.configuration, this.badRecordLogger);
-    List<FieldConverter> fieldConverterList = new ArrayList<>();
-    DictionaryClient client = createDictionaryClient();
-    dictClients.add(client);
-    String nullFormat =
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
-            .toString();
-    boolean isEmptyBadRecord = Boolean.parseBoolean(
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD)
-            .toString());
-    for (int i = 0; i < fields.length; i++) {
-      FieldConverter fieldConverter = null;
-      try {
-        fieldConverter = FieldEncoderFactory.getInstance().createFieldEncoder(fields[i], cache,
-            configuration.getTableIdentifier().getCarbonTableIdentifier(), i, nullFormat, client,
-            configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(),
-            localCaches[i], isEmptyBadRecord);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      fieldConverterList.add(fieldConverter);
-    }
-    converter.fieldConverters =
-        fieldConverterList.toArray(new FieldConverter[fieldConverterList.size()]);
-    converter.logHolder = new BadRecordLogHolder();
-    return converter;
-  }
-
-  @Override public int[] getCardinality() {
-    List<Integer> dimCardinality = new ArrayList<>();
-    if (fieldConverters != null) {
-      for (int i = 0; i < fieldConverters.length; i++) {
-        if (fieldConverters[i] instanceof AbstractDictionaryFieldConverterImpl) {
-          ((AbstractDictionaryFieldConverterImpl) fieldConverters[i])
-              .fillColumnCardinality(dimCardinality);
-        }
-      }
-    }
-    int[] cardinality = new int[dimCardinality.size()];
-    for (int i = 0; i < dimCardinality.size(); i++) {
-      cardinality[i] = dimCardinality.get(i);
-    }
-    return cardinality;
-  }
-}


[46/50] [abbrv] carbondata git commit: [CARBONDATA-1533] Fixed decimal data load fail issue and restricted max characters per column

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/ad25ffc3/integration/spark-common-test/src/test/resources/badrecords/bigtabbad.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/badrecords/bigtabbad.csv b/integration/spark-common-test/src/test/resources/badrecords/bigtabbad.csv
index 44d3dbe..d20fb7a 100644
--- a/integration/spark-common-test/src/test/resources/badrecords/bigtabbad.csv
+++ b/integration/spark-common-test/src/test/resources/badrecords/bigtabbad.csv
@@ -1,5 +1,2 @@
 ckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagesp
 intobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobea
 nsarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequ
 icklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccounts
 detectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectf
 luffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffily
 accordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordi
 ngtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithely
-ckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagesp
 intobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobea
 nsarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequ
 icklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccounts
 detectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectf
 luffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffily
 accordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordi
 ngtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpre
 ssrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressreque
 sckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckages
 pintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobe
 ansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpress
 accountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccount
 sdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetect
 fluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffil
 yaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblith
 elyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpr
 essrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequ
 esckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckage
 spintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequickl
 yexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpres
 saccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccoun
 tsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetec
 tfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithely
-ckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagesp
 intobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobea
 nsarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequ
 icklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccounts
 detectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectf
 luffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffily
 accordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordi
 ngtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpre
 ssrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressreque
 sckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckages
 pintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobe
 ansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpress
 accountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccount
 sdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetect
 fluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffil
 yaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithely
-ckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagesp
 intobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobea
 nsarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequ
 icklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccounts
 detectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectf
 luffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffily
 accordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordi
 ngtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithely
 ckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagesp
 intobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobea
 nsarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequ
 icklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccounts
 detectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectf
 luffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffily
 accordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordi
 ngtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpre
 ssrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressreque
 sckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckages
 pintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobe
 ansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpress
 accountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccount
 sdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetect
 fluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffil
 yaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithely,werr
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ad25ffc3/integration/spark-common-test/src/test/resources/decimal_int_range.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/decimal_int_range.csv b/integration/spark-common-test/src/test/resources/decimal_int_range.csv
new file mode 100644
index 0000000..890c8e0
--- /dev/null
+++ b/integration/spark-common-test/src/test/resources/decimal_int_range.csv
@@ -0,0 +1,4 @@
+d1
+111111
+222222.120
+333333.12345
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ad25ffc3/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
index 39167e2..937aee9 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
@@ -17,7 +17,11 @@
 
 package org.apache.carbondata.spark.testsuite.bigdecimal
 
+import java.math.BigDecimal
+
+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.spark.sql.test.util.QueryTest
@@ -32,6 +36,7 @@ class TestBigDecimal extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists hiveTable")
     sql("drop table if exists hiveBigDecimal")
     sql("drop table if exists carbonBigDecimal_2")
+    sql("DROP TABLE IF EXISTS decimal_int_test")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
     CarbonProperties.getInstance().addProperty(CarbonCommonConstants.SORT_SIZE, "1")
@@ -183,11 +188,22 @@ class TestBigDecimal extends QueryTest with BeforeAndAfterAll {
       sql("select avg(salary)/10 from hiveBigDecimal"))
   }
 
+  test("test decimal compression where both precision and data falls in integer range") {
+    sql("create table decimal_int_test(d1 decimal(9,3)) stored by 'carbondata'")
+    sql(s"load data inpath '$resourcesPath/decimal_int_range.csv' into table decimal_int_test")
+    sql("select * from decimal_int_test").show(false)
+    checkAnswer(sql("select * from decimal_int_test"),
+      Seq(Row(new BigDecimal("111111.000")),
+        Row(new BigDecimal("222222.120")),
+        Row(new BigDecimal("333333.123"))))
+  }
+
   override def afterAll {
     sql("drop table if exists carbonTable")
     sql("drop table if exists hiveTable")
     sql("drop table if exists hiveBigDecimal")
     sql("drop table if exists carbonBigDecimal_2")
+    sql("DROP TABLE IF EXISTS decimal_int_test")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
     CarbonProperties.getInstance().addProperty(CarbonCommonConstants.SORT_SIZE,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ad25ffc3/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
index e6db5e2..61646ec 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormat.java
@@ -72,6 +72,9 @@ public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWri
   public static final String NUMBER_OF_COLUMNS = "carbon.csvinputformat.number.of.columns";
   public static final int DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 2000;
   public static final int THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 20000;
+  // As Short data type is used for storing the length of a column during data processing hence
+  // the maximum characters that can be supported should be less than Short max value
+  public static final int MAX_CHARS_PER_COLUMN_DEFAULT = 32000;
 
   private static LogService LOGGER =
       LogServiceFactory.getLogService(CSVInputFormat.class.toString());
@@ -243,7 +246,7 @@ public class CSVInputFormat extends FileInputFormat<NullWritable, StringArrayWri
       parserSettings.setIgnoreLeadingWhitespaces(false);
       parserSettings.setIgnoreTrailingWhitespaces(false);
       parserSettings.setSkipEmptyLines(false);
-      parserSettings.setMaxCharsPerColumn(100000);
+      parserSettings.setMaxCharsPerColumn(MAX_CHARS_PER_COLUMN_DEFAULT);
       String maxColumns = job.get(MAX_COLUMNS);
       parserSettings.setMaxColumns(Integer.parseInt(maxColumns));
       parserSettings.getFormat().setQuote(job.get(QUOTE, QUOTE_DEFAULT).charAt(0));


[20/50] [abbrv] carbondata git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/carbondata

Posted by ja...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/carbondata


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

Branch: refs/heads/streaming_ingest
Commit: 663834aeb119aad1fc8e7933c4bf769963755c75
Parents: 6c73bee b221c98
Author: chenliang613 <ch...@apache.org>
Authored: Fri Sep 29 14:30:43 2017 +0800
Committer: chenliang613 <ch...@apache.org>
Committed: Fri Sep 29 14:30:43 2017 +0800

----------------------------------------------------------------------
 README.md                                       |    2 +-
 .../core/datamap/DataMapStoreManager.java       |  111 +-
 .../carbondata/core/datamap/TableDataMap.java   |   35 +-
 .../carbondata/core/datastore/TableSpec.java    |    4 +-
 .../page/UnsafeVarLengthColumnPage.java         |   12 -
 .../datastore/page/VarLengthColumnPageBase.java |    2 +-
 .../carbondata/core/indexstore/Blocklet.java    |   49 -
 .../core/indexstore/BlockletDetailsFetcher.java |   47 +
 .../core/indexstore/ExtendedBlocklet.java       |   82 ++
 .../TableBlockIndexUniqueIdentifier.java        |    4 +
 .../blockletindex/BlockletDataMap.java          |   25 +-
 .../blockletindex/BlockletDataMapFactory.java   |   71 +-
 .../ThriftWrapperSchemaConverterImpl.java       |    4 +-
 .../core/metadata/schema/BucketingInfo.java     |   32 +-
 .../core/metadata/schema/PartitionInfo.java     |   95 +-
 .../core/metadata/schema/table/TableSchema.java |   25 +
 .../scan/executor/util/RestructureUtil.java     |    7 +-
 .../carbondata/core/scan/filter/FilterUtil.java |   12 +-
 .../executer/RowLevelFilterExecuterImpl.java    |   27 +-
 format/src/main/thrift/schema.thrift            |    4 +-
 .../hadoop/api/CarbonTableInputFormat.java      |   46 +-
 .../carbondata/hadoop/api/DataMapJob.java       |    5 +-
 .../hadoop/api/DistributableDataMapFormat.java  |   14 +-
 .../src/test/resources/partition_data.csv       |    1 -
 .../testsuite/datamap/DataMapWriterSuite.scala  |    4 +-
 .../iud/UpdateCarbonTableTestCase.scala         |  804 +++++------
 .../carbondata/spark/load/CarbonLoaderUtil.java |   23 +-
 .../load/DataLoadProcessorStepOnSpark.scala     |    2 +-
 .../carbondata/spark/rdd/SparkDataMapJob.scala  |   12 +-
 .../carbondata/spark/util/CommonUtil.scala      |   24 +
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |    5 +-
 .../apache/spark/sql/hive/CarbonMetastore.scala |    6 +-
 .../VectorizedCarbonRecordReader.java           |    4 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   20 +-
 .../spark/sql/CarbonCatalystOperators.scala     |   16 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |    4 +-
 .../sql/CarbonDatasourceHadoopRelation.scala    |    4 +-
 .../org/apache/spark/sql/CarbonSource.scala     |    2 +-
 .../spark/sql/SparkUnknownExpression.scala      |    1 -
 .../execution/CarbonLateDecodeStrategy.scala    |  553 --------
 .../execution/command/AlterTableCommands.scala  |  460 ------
 .../command/CarbonCreateTableCommand.scala      |  107 ++
 .../CarbonDescribeFormattedCommand.scala        |  133 ++
 .../command/CarbonDropTableCommand.scala        |   94 ++
 .../sql/execution/command/DDLStrategy.scala     |  143 --
 .../sql/execution/command/IUDCommands.scala     |  857 ------------
 .../execution/command/carbonTableSchema.scala   | 1315 ------------------
 .../AlterTableCompactionCommand.scala           |   90 ++
 .../management/CarbonShowLoadsCommand.scala     |   50 +
 .../command/management/CleanFilesCommand.scala  |   58 +
 .../management/DeleteLoadByIdCommand.scala      |   48 +
 .../DeleteLoadByLoadDateCommand.scala           |   50 +
 .../management/LoadTableByInsertCommand.scala   |   53 +
 .../command/management/LoadTableCommand.scala   |  520 +++++++
 .../command/mutation/DeleteExecution.scala      |  322 +++++
 .../command/mutation/HorizontalCompaction.scala |  246 ++++
 .../HorizontalCompactionException.scala         |   24 +
 .../mutation/ProjectForDeleteCommand.scala      |  105 ++
 .../mutation/ProjectForUpdateCommand.scala      |  228 +++
 .../spark/sql/execution/command/package.scala   |   53 +
 .../AlterTableDropCarbonPartitionCommand.scala  |  176 +++
 .../AlterTableSplitCarbonPartitionCommand.scala |  187 +++
 .../partition/ShowCarbonPartitionsCommand.scala |   60 +
 .../schema/AlterTableAddColumnCommand.scala     |  115 ++
 .../AlterTableDataTypeChangeCommand.scala       |  116 ++
 .../schema/AlterTableDropColumnCommand.scala    |  148 ++
 .../schema/AlterTableRenameTableCommand.scala   |  174 +++
 .../strategy/CarbonLateDecodeStrategy.scala     |  554 ++++++++
 .../sql/execution/strategy/DDLStrategy.scala    |  162 +++
 .../spark/sql/hive/CarbonAnalysisRules.scala    |   10 +-
 .../spark/sql/hive/CarbonFileMetastore.scala    |    2 +-
 .../spark/sql/hive/CarbonHiveMetaStore.scala    |    2 +-
 .../spark/sql/hive/CarbonSessionState.scala     |    7 +-
 .../execution/command/CarbonHiveCommands.scala  |    5 +-
 .../sql/optimizer/CarbonLateDecodeRule.scala    |    3 +-
 .../sql/parser/CarbonSpark2SqlParser.scala      |   41 +-
 .../spark/sql/parser/CarbonSparkSqlParser.scala |    4 +-
 .../org/apache/spark/util/Compaction.scala      |    5 +-
 .../org/apache/spark/util/TableLoader.scala     |    4 +-
 .../partition/TestAlterPartitionTable.scala     |   69 +-
 .../vectorreader/AddColumnTestCases.scala       |   24 +
 .../vectorreader/VectorReaderTestCase.scala     |    2 +-
 .../newflow/sort/SortStepRowUtil.java           |    3 +-
 .../UnsafeSingleThreadFinalSortFilesMerger.java |    2 +-
 .../CarbonRowDataWriterProcessorStepImpl.java   |    2 +-
 .../steps/DataConverterProcessorStepImpl.java   |    8 +-
 86 files changed, 5058 insertions(+), 3983 deletions(-)
----------------------------------------------------------------------



[21/50] [abbrv] carbondata git commit: [RELEASE] Updated new version number 1.3.0-SNAPSHOT in master branch pom files

Posted by ja...@apache.org.
[RELEASE] Updated new version number 1.3.0-SNAPSHOT in master branch pom files


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

Branch: refs/heads/streaming_ingest
Commit: b9f10da6fa3eef48dba2934322a7073b1077e9c4
Parents: 663834a 87e45a4
Author: chenliang613 <ch...@apache.org>
Authored: Fri Sep 29 14:34:32 2017 +0800
Committer: chenliang613 <ch...@apache.org>
Committed: Fri Sep 29 14:34:32 2017 +0800

----------------------------------------------------------------------
 assembly/pom.xml                              | 2 +-
 common/pom.xml                                | 2 +-
 core/pom.xml                                  | 2 +-
 examples/flink/pom.xml                        | 2 +-
 examples/spark/pom.xml                        | 2 +-
 examples/spark2/pom.xml                       | 2 +-
 format/pom.xml                                | 2 +-
 hadoop/pom.xml                                | 2 +-
 integration/hive/pom.xml                      | 2 +-
 integration/presto/pom.xml                    | 2 +-
 integration/spark-common-cluster-test/pom.xml | 2 +-
 integration/spark-common-test/pom.xml         | 2 +-
 integration/spark-common/pom.xml              | 2 +-
 integration/spark/pom.xml                     | 2 +-
 integration/spark2/pom.xml                    | 2 +-
 pom.xml                                       | 2 +-
 processing/pom.xml                            | 2 +-
 17 files changed, 17 insertions(+), 17 deletions(-)
----------------------------------------------------------------------



[50/50] [abbrv] carbondata git commit: [CARBONDATA-1174] Streaming Ingestion - schema validation and streaming examples

Posted by ja...@apache.org.
[CARBONDATA-1174] Streaming Ingestion - schema validation and streaming examples

1.schema validation of input data if its from a file source when schema is specified.
2.added streaming examples - for file stream and socket stream sources

This closes #1352


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

Branch: refs/heads/streaming_ingest
Commit: 08a82b59b409ca3e1881e9da7bc6d8a760b1d0d4
Parents: 441907e
Author: Aniket Adnaik <an...@gmail.com>
Authored: Thu Jun 15 11:57:43 2017 -0700
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Oct 10 11:06:52 2017 +0800

----------------------------------------------------------------------
 .../streaming/CarbonStreamingCommitInfo.java    |   6 +-
 ...CarbonStreamingIngestFileSourceExample.scala | 146 +++++++++++++
 ...rbonStreamingIngestSocketSourceExample.scala | 160 ++++++++++++++
 .../examples/utils/StreamingExampleUtil.scala   | 145 +++++++++++++
 .../org/apache/spark/sql/CarbonSource.scala     | 210 +++++++++++++++++--
 .../CarbonStreamingOutpurWriteFactory.scala     |  88 --------
 .../CarbonStreamingOutputWriteFactory.scala     |  88 ++++++++
 .../CarbonSourceSchemaValidationTest.scala      |  61 ++++++
 8 files changed, 801 insertions(+), 103 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/08a82b59/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java b/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java
index 6cf303a..2027566 100644
--- a/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java
@@ -36,7 +36,7 @@ public class CarbonStreamingCommitInfo {
 
   private long batchID;
 
-  private String fileOffset;
+  private long fileOffset;
 
   private long transactionID;     // future use
 
@@ -67,6 +67,8 @@ public class CarbonStreamingCommitInfo {
     this.batchID = batchID;
 
     this.transactionID = -1;
+
+    this.fileOffset = 0;
   }
 
   public String getDataBase() {
@@ -93,7 +95,7 @@ public class CarbonStreamingCommitInfo {
     return batchID;
   }
 
-  public String getFileOffset() {
+  public long getFileOffset() {
     return fileOffset;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/08a82b59/examples/spark2/src/main/scala/org/apache/carbondata/examples/streaming/CarbonStreamingIngestFileSourceExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/streaming/CarbonStreamingIngestFileSourceExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/streaming/CarbonStreamingIngestFileSourceExample.scala
new file mode 100644
index 0000000..ebe0a5c
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/streaming/CarbonStreamingIngestFileSourceExample.scala
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.spark.sql.{SaveMode, SparkSession}
+import org.apache.spark.sql.streaming.ProcessingTime
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.utils.{StreamingExampleUtil}
+
+/**
+ * Covers spark structured streaming scenario where user streams data
+ * from a file source (input source) and write into carbondata table(output sink).
+ * This example uses csv file as a input source and writes
+ * into target carbon table. The target carbon table must exist.
+ */
+
+object CarbonStreamingIngestFileSourceExample {
+
+  def main(args: Array[String]) {
+
+    val rootPath = new File(this.getClass.getResource("/").getPath
+      + "../../../..").getCanonicalPath
+    val storeLocation = s"$rootPath/examples/spark2/target/store"
+    val warehouse = s"$rootPath/examples/spark2/target/warehouse"
+    val metastoredb = s"$rootPath/examples/spark2/target"
+    val csvDataDir = s"$rootPath/examples/spark2/resources/csvDataDir"
+    val streamTableName = s"_carbon_file_stream_table_"
+    val streamTablePath = s"$storeLocation/default/$streamTableName"
+    val ckptLocation = s"$rootPath/examples/spark2/resources/ckptDir"
+
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+
+    // cleanup residual files, if any
+    StreamingExampleUtil.cleanUpDir(csvDataDir, ckptLocation)
+
+    import org.apache.spark.sql.CarbonSession._
+    val spark = SparkSession
+      .builder()
+      .master("local")
+      .appName("CarbonFileStreamingExample")
+      .config("spark.sql.warehouse.dir", warehouse)
+      .getOrCreateCarbonSession(storeLocation, metastoredb)
+
+    spark.sparkContext.setLogLevel("ERROR")
+
+    // Writes Dataframe to CarbonData file:
+    import spark.implicits._
+    import org.apache.spark.sql.types._
+    // drop table if exists previously
+    spark.sql(s"DROP TABLE IF EXISTS $streamTableName")
+    // Create target carbon table
+    spark.sql(
+      s"""
+         | CREATE TABLE $streamTableName(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT
+         | )
+         | STORED BY 'carbondata'""".stripMargin)
+
+    // Generate CSV data and write to CSV file
+    StreamingExampleUtil.generateCSVDataFile(spark, 1, csvDataDir, SaveMode.Overwrite)
+
+    // scalastyle:off
+    spark.sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '$csvDataDir'
+         | INTO TABLE $streamTableName
+         | OPTIONS('FILEHEADER'='id,name,city,salary'
+         | )""".stripMargin)
+    // scalastyle:on
+
+    // check initial table data
+    spark.sql(s""" SELECT * FROM $streamTableName """).show()
+
+    // define custom schema
+    val inputSchema = new StructType().
+      add("id", "integer").
+      add("name", "string").
+      add("city", "string").
+      add("salary", "float")
+
+    // setup csv file as a input streaming source
+    val csvReadDF = spark.readStream.
+      format("csv").
+      option("sep", ",").
+      schema(inputSchema).
+      option("path", csvDataDir).
+      option("header", "true").
+      load()
+
+    // Write data from csv format streaming source to carbondata target format
+    // set trigger to every 1 second
+    val qry = csvReadDF.writeStream
+      .format("carbondata")
+      .trigger(ProcessingTime("1 seconds"))
+      .option("checkpointLocation", ckptLocation)
+      .option("path", streamTablePath)
+      .start()
+
+    // In a separate thread append data every 2 seconds to existing csv
+    val gendataThread: Thread = new Thread() {
+      override def run(): Unit = {
+        for (i <- 1 to 5) {
+          Thread.sleep(2)
+          StreamingExampleUtil.
+            generateCSVDataFile(spark, i * 10 + 1, csvDataDir, SaveMode.Append)
+        }
+      }
+    }
+    gendataThread.start()
+    gendataThread.join()
+
+    // stop streaming execution after 5 sec delay
+    Thread.sleep(5000)
+    qry.stop()
+
+    // verify streaming data is added into the table
+    spark.sql(s""" SELECT * FROM $streamTableName """).show()
+
+    // Cleanup residual files and table data
+    StreamingExampleUtil.cleanUpDir(csvDataDir, ckptLocation)
+    spark.sql(s"DROP TABLE IF EXISTS $streamTableName")
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/08a82b59/examples/spark2/src/main/scala/org/apache/carbondata/examples/streaming/CarbonStreamingIngestSocketSourceExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/streaming/CarbonStreamingIngestSocketSourceExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/streaming/CarbonStreamingIngestSocketSourceExample.scala
new file mode 100644
index 0000000..bbf7ef2
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/streaming/CarbonStreamingIngestSocketSourceExample.scala
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.spark.sql.{SaveMode, SparkSession}
+import org.apache.spark.sql.streaming.ProcessingTime
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.examples.utils.StreamingExampleUtil
+
+
+/**
+ * This example reads stream data from socket source (input) and write into
+ * existing carbon table(output).
+ *
+ * It uses localhost and port (9999) to create a socket and write to it.
+ * Exmaples uses two threads one to write data to socket and other thread
+ * to receive data from socket and write into carbon table.
+ */
+
+// scalastyle:off println
+object CarbonStreamingIngestSocketSourceExample {
+
+  def main(args: Array[String]) {
+
+    // setup localhost and port number
+    val host = "localhost"
+    val port = 9999
+    // setup paths
+    val rootPath = new File(this.getClass.getResource("/").getPath
+      + "../../../..").getCanonicalPath
+    val storeLocation = s"$rootPath/examples/spark2/target/store"
+    val warehouse = s"$rootPath/examples/spark2/target/warehouse"
+    val metastoredb = s"$rootPath/examples/spark2/target"
+    val csvDataDir = s"$rootPath/examples/spark2/resources/csvDataDir"
+    val streamTableName = s"_carbon_socket_stream_table_"
+    val streamTablePath = s"$storeLocation/default/$streamTableName"
+    val ckptLocation = s"$rootPath/examples/spark2/resources/ckptDir"
+
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+
+    // cleanup residual files, if any
+    StreamingExampleUtil.cleanUpDir(csvDataDir, ckptLocation)
+
+    import org.apache.spark.sql.CarbonSession._
+    val spark = SparkSession
+      .builder()
+      .master("local[2]")
+      .appName("CarbonNetworkStreamingExample")
+      .config("spark.sql.warehouse.dir", warehouse)
+      .getOrCreateCarbonSession(storeLocation, metastoredb)
+
+    spark.sparkContext.setLogLevel("ERROR")
+
+    // Writes Dataframe to CarbonData file:
+    import spark.implicits._
+
+    // drop table if exists previously
+    spark.sql(s"DROP TABLE IF EXISTS ${streamTableName}")
+
+    // Create target carbon table and populate with initial data
+    spark.sql(
+      s"""
+         | CREATE TABLE ${streamTableName}(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT
+         | )
+         | STORED BY 'carbondata'""".stripMargin)
+
+    // Generate CSV data and write to CSV file
+    StreamingExampleUtil.generateCSVDataFile(spark, 1, csvDataDir, SaveMode.Overwrite)
+
+    // load the table
+    // scalastyle:off
+    spark.sql(
+      s"""
+         | LOAD DATA LOCAL INPATH '$csvDataDir'
+         | INTO TABLE ${streamTableName}
+         | OPTIONS('FILEHEADER'='id,name,city,salary'
+         | )""".stripMargin)
+
+
+    spark.sql(s""" SELECT * FROM ${streamTableName} """).show()
+
+    // Create server socket in main thread
+    val serverSocket = StreamingExampleUtil.createserverSocket(host, port)
+
+    // Start client thread to receive streaming data and write into carbon
+    val streamWriterThread: Thread = new Thread() {
+      override def run(): Unit= {
+
+        try {
+          // Setup read stream to read input data from socket
+          val readSocketDF = spark.readStream
+            .format("socket")
+            .option("host", host)
+            .option("port", port)
+            .load()
+
+          // Write data from socket stream to carbondata file
+          val qry = readSocketDF.writeStream
+            .format("carbondata")
+            .trigger(ProcessingTime("2 seconds"))
+            .option("checkpointLocation", ckptLocation)
+            .option("path", streamTablePath)
+            .start()
+
+          qry.awaitTermination()
+        } catch {
+          case e: InterruptedException => println("Done reading and writing streaming data")
+        }
+      }
+    }
+    streamWriterThread.start()
+
+    // wait for client to connection request and accept
+    val clientSocket = StreamingExampleUtil.waitToForClientConnection(serverSocket.get)
+
+    // Write to client's connected socket every 2 seconds, for 5 times
+    StreamingExampleUtil.writeToSocket(clientSocket, 5, 2, 11)
+
+    Thread.sleep(2000)
+    // interrupt client thread to stop streaming query
+    streamWriterThread.interrupt()
+    //wait for client thread to finish
+    streamWriterThread.join()
+
+    //Close the server socket
+    serverSocket.get.close()
+
+    // verify streaming data is added into the table
+    // spark.sql(s""" SELECT * FROM ${streamTableName} """).show()
+
+    // Cleanup residual files and table data
+    StreamingExampleUtil.cleanUpDir(csvDataDir, ckptLocation)
+    spark.sql(s"DROP TABLE IF EXISTS ${streamTableName}")
+  }
+}
+// scalastyle:on println

http://git-wip-us.apache.org/repos/asf/carbondata/blob/08a82b59/examples/spark2/src/main/scala/org/apache/carbondata/examples/utils/StreamingExampleUtil.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/utils/StreamingExampleUtil.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/utils/StreamingExampleUtil.scala
new file mode 100644
index 0000000..6eab491
--- /dev/null
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/utils/StreamingExampleUtil.scala
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.utils
+
+import java.io.{IOException, PrintWriter}
+import java.net.{ServerSocket, Socket}
+
+import scala.tools.nsc.io.Path
+
+import org.apache.spark.sql.{DataFrame, SaveMode, SparkSession}
+
+
+
+/**
+ * Utility functions for streaming ingest examples
+ */
+
+// scalastyle:off println
+object StreamingExampleUtil {
+
+  // Clean up directories recursively, accepts variable arguments
+  def cleanUpDir(dirPaths: String*): Unit = {
+
+      // if (args.length < 1) {
+    if (dirPaths.size < 1) {
+      System.err.println("Usage: StreamingCleanupUtil <dirPath> [dirpath]...")
+      System.exit(1)
+    }
+
+    var i = 0
+    while (i < dirPaths.size) {
+      try {
+        val path: Path = Path(dirPaths(i))
+        path.deleteRecursively()
+      } catch {
+        case ioe: IOException => println("IO Exception while deleting files recursively" + ioe)
+      }
+      i = i + 1
+    }
+  }
+
+  // Generates csv data and write to csv files at given path
+  def generateCSVDataFile(spark: SparkSession,
+                        idStart: Int,
+                        csvDirPath: String,
+                        saveMode: SaveMode): Unit = {
+    // Create csv data frame file
+    val csvRDD = spark.sparkContext.parallelize(1 to 10)
+      .map(id => (id, "name_ABC", "city_XYZ", 10000.00*id))
+      val csvDataDF = spark.createDataFrame(csvRDD).toDF("id", "name", "city", "salary")
+
+
+    csvDataDF.write
+      .option("header", "false")
+      .mode(saveMode)
+      .csv(csvDirPath)
+  }
+
+  // Generates csv data frame and returns to caller
+  def generateCSVDataDF(spark: SparkSession,
+                        idStart: Int): DataFrame = {
+    // Create csv data frame file
+    val csvRDD = spark.sparkContext.parallelize(1 to 10)
+      .map(id => (id, "name_ABC", "city_XYZ", 10000.00*id))
+    val csvDataDF = spark.createDataFrame(csvRDD).toDF("id", "name", "city", "salary")
+    csvDataDF
+  }
+
+  // Create server socket for socket streaming source
+  def createserverSocket(host: String, port: Int): Option[ServerSocket] = {
+    try {
+      Some(new ServerSocket(port))
+    } catch {
+      case e: java.net.ConnectException =>
+        println("Error Connecting to" + host + ":" + port, e)
+        None
+    }
+  }
+
+  // Create server socket for socket streaming source
+  def waitToForClientConnection(serverSocket: ServerSocket): Socket = {
+    serverSocket.accept()
+  }
+
+  // Create server socket for socket streaming source
+  def closeServerSocket(serverSocket: ServerSocket): Unit = {
+    serverSocket.close()
+  }
+
+  // write periodically on given socket
+  def writeToSocket(clientSocket: Socket,
+                   iterations: Int,
+                   delay: Int,
+                   startID: Int): Unit = {
+
+    var nItr = 10
+    var nDelay = 5
+
+    // iterations range check
+    if (iterations >= 1 || iterations <= 50) {
+      nItr = iterations
+    } else {
+      println("Number of iterations exceeds limit. Setting to default 10 iterations")
+    }
+
+    // delay range check (1 second to 60 seconds)
+    if (delay >= 1 || delay <= 60) {
+      nDelay = delay
+    } else {
+      println("Delay exceeds the limit. Setting it to default 2 seconds")
+    }
+
+    val socketWriter = new PrintWriter(clientSocket.getOutputStream())
+
+    var j = startID
+
+    for (i <- startID to startID + nItr) {
+      // write 5 records per iteration
+      for (id <- j to j + 5 ) {
+        socketWriter.println(id.toString + ", name_" + i
+          + ", city_" + i + ", " + (i*10000.00).toString)
+      }
+      j = j + 5
+      socketWriter.flush()
+      Thread.sleep(nDelay*1000)
+    }
+    socketWriter.close()
+  }
+}
+// scalastyle:on println

http://git-wip-us.apache.org/repos/asf/carbondata/blob/08a82b59/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 d496de2..6eacb19 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
@@ -17,14 +17,20 @@
 
 package org.apache.spark.sql
 
+import java.io.{BufferedWriter, FileWriter, IOException}
+import java.util.UUID
+
 import scala.collection.JavaConverters._
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
 import scala.language.implicitConversions
 
 import org.apache.commons.lang.StringUtils
 import org.apache.hadoop.fs.{FileStatus, Path}
 import org.apache.hadoop.mapreduce.Job
+import org.apache.parquet.schema.InvalidSchemaException
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
 import org.apache.spark.sql.execution.command.{TableModel, TableNewProcessor}
 import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory}
 import org.apache.spark.sql.execution.strategy.CarbonLateDecodeStrategy
@@ -33,10 +39,12 @@ import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 import org.apache.spark.sql.sources._
 import org.apache.spark.sql.streaming.CarbonStreamingOutputWriterFactory
-import org.apache.spark.sql.types.{StringType, StructType}
+import org.apache.spark.sql.types._
 
+import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 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}
@@ -44,7 +52,6 @@ import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.spark.CarbonOption
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
-
 /**
  * Carbon relation provider compliant to data source api.
  * Creates carbon relations
@@ -52,7 +59,11 @@ import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 class CarbonSource extends CreatableRelationProvider with RelationProvider
   with SchemaRelationProvider with DataSourceRegister with FileFormat  {
 
-  override def shortName(): String = "carbondata"
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  override def shortName(): String = {
+    "carbondata"
+  }
 
   // will be called if hive supported create table command is provided
   override def createRelation(sqlContext: SQLContext,
@@ -181,8 +192,7 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
 
   /**
    * Returns the path of the table
-   *
-     * @param sparkSession
+   * @param sparkSession
    * @param dbName
    * @param tableName
    * @return
@@ -217,22 +227,196 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
    * be put here.  For example, user defined output committer can be configured here
    * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass.
    */
-  def prepareWrite(
+  override def prepareWrite(
     sparkSession: SparkSession,
     job: Job,
     options: Map[String, String],
-    dataSchema: StructType): OutputWriterFactory = new CarbonStreamingOutputWriterFactory()
+    dataSchema: StructType): OutputWriterFactory = {
+
+    // Check if table with given path exists
+    // validateTable(options.get("path").get)
+    validateTable(options("path"))
+
+    /* Check id streaming data schema matches with carbon table schema
+     * Data from socket source does not have schema attached to it,
+     * Following check is to ignore schema validation for socket source.
+     */
+    if (!(dataSchema.size.equals(1) &&
+      dataSchema.fields(0).dataType.equals(StringType))) {
+      val path = options.get("path")
+      val tablePath: String = path match {
+        case Some(value) => value
+        case None => ""
+      }
+
+      val carbonTableSchema: org.apache.carbondata.format.TableSchema =
+        getTableSchema(sparkSession: SparkSession, tablePath: String)
+      val isSchemaValid = validateSchema(carbonTableSchema, dataSchema)
+
+      if(!isSchemaValid) {
+        LOGGER.error("Schema Validation Failed: streaming data schema"
+          + "does not match with carbon table schema")
+        throw new InvalidSchemaException("Schema Validation Failed : " +
+          "streaming data schema does not match with carbon table schema")
+      }
+    }
+    new CarbonStreamingOutputWriterFactory()
+  }
 
   /**
-   * When possible, this method should return the schema of the given `files`.  When the format
-   * does not support inference, or no valid files are given should return None.  In these cases
-   * Spark will require that user specify the schema manually.
+   * Read schema from existing carbon table
+   * @param sparkSession
+   * @param tablePath carbon table path
+   * @return TableSchema read from provided table path
    */
-  def inferSchema(
+  private def getTableSchema(
+    sparkSession: SparkSession,
+    tablePath: String): org.apache.carbondata.format.TableSchema = {
+
+    val formattedTablePath = tablePath.replace('\\', '/')
+    val names = formattedTablePath.split("/")
+    if (names.length < 3) {
+      throw new IllegalArgumentException("invalid table path: " + tablePath)
+    }
+    val tableName : String = names(names.length - 1)
+    val dbName : String = names(names.length - 2)
+    val storePath = formattedTablePath.substring(0,
+      formattedTablePath.lastIndexOf
+      (dbName.concat(CarbonCommonConstants.FILE_SEPARATOR)
+        .concat(tableName)) - 1)
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val thriftTableInfo: org.apache.carbondata.format.TableInfo =
+      metastore.getThriftTableInfo(new CarbonTablePath(storePath, dbName, tableName))(sparkSession)
+
+    val factTable: org.apache.carbondata.format.TableSchema = thriftTableInfo.getFact_table
+    factTable
+  }
+
+  /**
+   * Validates streamed schema against existing table schema
+   * @param carbonTableSchema existing carbon table schema
+   * @param dataSchema streamed data schema
+   * @return true if schema validation is successful else false
+   */
+  private def validateSchema(
+      carbonTableSchema: org.apache.carbondata.format.TableSchema,
+      dataSchema: StructType): Boolean = {
+
+    val columnnSchemaValues = carbonTableSchema.getTable_columns
+      .asScala.sortBy(_.schemaOrdinal)
+
+    var columnDataTypes = new ListBuffer[String]()
+    columnnSchemaValues.foreach(columnDataType =>
+      columnDataTypes.append(columnDataType.data_type.toString))
+    val tableColumnDataTypeList = columnDataTypes.toList
+
+    var streamSchemaDataTypes = new ListBuffer[String]()
+    dataSchema.fields.foreach(item => streamSchemaDataTypes
+      .append(mapStreamingDataTypeToString(item.dataType.toString)))
+    val streamedDataTypeList = streamSchemaDataTypes.toList
+
+    val isValid = tableColumnDataTypeList == streamedDataTypeList
+    isValid
+  }
+
+  /**
+   * Maps streamed datatype to carbon datatype
+   * @param dataType
+   * @return String
+   */
+  def mapStreamingDataTypeToString(dataType: String): String = {
+    import org.apache.carbondata.format.DataType
+    dataType match {
+      case "IntegerType" => DataType.INT.toString
+      case "StringType" => DataType.STRING.toString
+      case "DateType" => DataType.DATE.toString
+      case "DoubleType" => DataType.DOUBLE.toString
+      case "FloatType" => DataType.DOUBLE.toString
+      case "LongType" => DataType.LONG.toString
+      case "ShortType" => DataType.SHORT.toString
+      case "TimestampType" => DataType.TIMESTAMP.toString
+    }
+  }
+
+  /**
+   * Validates if given table exists or throws exception
+   * @param tablePath existing carbon table path
+   * @return None
+   */
+  private def validateTable(tablePath: String): Unit = {
+
+    val formattedTablePath = tablePath.replace('\\', '/')
+    val names = formattedTablePath.split("/")
+    if (names.length < 3) {
+      throw new IllegalArgumentException("invalid table path: " + tablePath)
+    }
+    val tableName : String = names(names.length - 1)
+    val dbName : String = names(names.length - 2)
+    val storePath = formattedTablePath.substring(0,
+      formattedTablePath.lastIndexOf
+      (((dbName.concat(CarbonCommonConstants.FILE_SEPARATOR).toString)
+        .concat(tableName)).toString) - 1)
+    val absoluteTableIdentifier: AbsoluteTableIdentifier =
+      new AbsoluteTableIdentifier(storePath,
+        new CarbonTableIdentifier(dbName, tableName,
+          UUID.randomUUID().toString))
+
+    if (!checkIfTableExists(absoluteTableIdentifier)) {
+      throw new NoSuchTableException(dbName, tableName)
+    }
+  }
+
+  /**
+   * Checks if table exists by checking its schema file
+   * @param absoluteTableIdentifier
+   * @return Boolean
+   */
+  private def checkIfTableExists(absoluteTableIdentifier: AbsoluteTableIdentifier): Boolean = {
+    val carbonTablePath: CarbonTablePath = CarbonStorePath
+      .getCarbonTablePath(absoluteTableIdentifier)
+    val schemaFilePath: String = carbonTablePath.getSchemaFilePath
+    FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.LOCAL) ||
+      FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.HDFS) ||
+      FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.VIEWFS)
+  }
+
+  /**
+   * If user wants to stream data from carbondata table source
+   * and if following conditions are true:
+   *    1. No schema provided by the user in readStream()
+   *    2. spark.sql.streaming.schemaInference is set to true
+   * carbondata can infer table schema from a valid table path
+   * The schema inference is not mandatory, but good have.
+   * When possible, this method should return the schema of the given `files`.
+   * If the format does not support schema inference, or no valid files
+   * are given it should return None. In these cases Spark will require that
+   * user specify the schema manually.
+   */
+  override def inferSchema(
     sparkSession: SparkSession,
     options: Map[String, String],
-    files: Seq[FileStatus]): Option[StructType] = Some(new StructType().add("value", StringType))
+    files: Seq[FileStatus]): Option[StructType] = {
+
+    val path = options.get("path")
+    val tablePath: String = path match {
+      case Some(value) => value
+      case None => ""
+    }
+    // Check if table with given path exists
+    validateTable(tablePath)
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val carbonTableSchema: org.apache.carbondata.format.TableSchema =
+      getTableSchema(sparkSession: SparkSession, tablePath: String)
+    val columnnSchemaValues = carbonTableSchema.getTable_columns
+      .asScala.sortBy(_.schemaOrdinal)
+
+    var structFields = new ArrayBuffer[StructField]()
+    columnnSchemaValues.foreach(columnSchema =>
+      structFields.append(StructField(columnSchema.column_name,
+        CatalystSqlParser.parseDataType(columnSchema.data_type.toString), true)))
 
+    Some(new StructType(structFields.toArray))
+  }
 }
 
 object CarbonSource {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/08a82b59/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutpurWriteFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutpurWriteFactory.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutpurWriteFactory.scala
deleted file mode 100644
index be69885..0000000
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutpurWriteFactory.scala
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements.  See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.spark.sql.streaming
-
-
-import java.util.concurrent.ConcurrentHashMap
-
-import org.apache.hadoop.mapreduce.TaskAttemptContext
-import org.apache.spark.sql.execution.datasources.OutputWriterFactory
-import org.apache.spark.sql.types.StructType
-
-import org.apache.carbondata.core.util.path.CarbonTablePath
-
-
-class CarbonStreamingOutputWriterFactory extends OutputWriterFactory {
-
- /**
-  * When writing to a [[org.apache.spark.sql.execution.datasources.HadoopFsRelation]],
-  * this method gets called by each task on executor side
-  * to instantiate new [[org.apache.spark.sql.execution.datasources.OutputWriter]]s.
-  *
-  * @param path Path to write the file.
-  * @param dataSchema Schema of the rows to be written. Partition columns are not
-  *                   included in the schema if the relation being written is
-  *                   partitioned.
-  * @param context The Hadoop MapReduce task context.
-  */
-
-  override def newInstance(
-    path: String,
-
-    dataSchema: StructType,
-
-    context: TaskAttemptContext) : CarbonStreamingOutputWriter = {
-
-        new CarbonStreamingOutputWriter(path, context)
-  }
-
-  override def getFileExtension(context: TaskAttemptContext): String = {
-
-    CarbonTablePath.STREAM_FILE_NAME_EXT
-  }
-
-}
-
-object CarbonStreamingOutpurWriterFactory {
-
-  private[this] val writers = new ConcurrentHashMap[String, CarbonStreamingOutputWriter]()
-
-  def addWriter(path: String, writer: CarbonStreamingOutputWriter): Unit = {
-
-    if (writers.contains(path)) {
-      throw new IllegalArgumentException(path + "writer already exists")
-    }
-
-    writers.put(path, writer)
-  }
-
-  def getWriter(path: String): CarbonStreamingOutputWriter = {
-
-    writers.get(path)
-  }
-
-  def containsWriter(path: String): Boolean = {
-
-    writers.containsKey(path)
-  }
-
-  def removeWriter(path: String): Unit = {
-
-    writers.remove(path)
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/08a82b59/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutputWriteFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutputWriteFactory.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutputWriteFactory.scala
new file mode 100644
index 0000000..c5e4226
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutputWriteFactory.scala
@@ -0,0 +1,88 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.streaming
+
+
+import java.util.concurrent.ConcurrentHashMap
+
+import org.apache.hadoop.mapreduce.TaskAttemptContext
+import org.apache.spark.sql.execution.datasources.OutputWriterFactory
+import org.apache.spark.sql.types.StructType
+
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+
+class CarbonStreamingOutputWriterFactory extends OutputWriterFactory {
+
+ /**
+  * When writing to a [[org.apache.spark.sql.execution.datasources.HadoopFsRelation]],
+  * this method gets called by each task on executor side
+  * to instantiate new [[org.apache.spark.sql.execution.datasources.OutputWriter]]s.
+  *
+  * @param path Path to write the file.
+  * @param dataSchema Schema of the rows to be written. Partition columns are not
+  *                   included in the schema if the relation being written is
+  *                   partitioned.
+  * @param context The Hadoop MapReduce task context.
+  */
+
+  override def newInstance(
+    path: String,
+
+    dataSchema: StructType,
+
+    context: TaskAttemptContext) : CarbonStreamingOutputWriter = {
+
+        new CarbonStreamingOutputWriter(path, context)
+  }
+
+  override def getFileExtension(context: TaskAttemptContext): String = {
+
+    CarbonTablePath.STREAM_FILE_NAME_EXT
+  }
+
+}
+
+object CarbonStreamingOutputWriterFactory {
+
+  private[this] val writers = new ConcurrentHashMap[String, CarbonStreamingOutputWriter]()
+
+  def addWriter(path: String, writer: CarbonStreamingOutputWriter): Unit = {
+
+    if (writers.contains(path)) {
+      throw new IllegalArgumentException(path + "writer already exists")
+    }
+
+    writers.put(path, writer)
+  }
+
+  def getWriter(path: String): CarbonStreamingOutputWriter = {
+
+    writers.get(path)
+  }
+
+  def containsWriter(path: String): Boolean = {
+
+    writers.containsKey(path)
+  }
+
+  def removeWriter(path: String): Unit = {
+
+    writers.remove(path)
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/08a82b59/integration/spark2/src/test/scala/org/apache/spark/carbondata/streaming/CarbonSourceSchemaValidationTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/streaming/CarbonSourceSchemaValidationTest.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/streaming/CarbonSourceSchemaValidationTest.scala
new file mode 100644
index 0000000..f00eea5
--- /dev/null
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/streaming/CarbonSourceSchemaValidationTest.scala
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.carbondata.streaming
+
+import org.apache.hadoop.mapreduce.Job
+import org.apache.spark.sql.common.util.Spark2QueryTest
+import org.apache.spark.sql.{CarbonSource, SparkSession}
+import org.apache.spark.sql.streaming.CarbonStreamingOutputWriterFactory
+import org.apache.spark.sql.test.TestQueryExecutor
+import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType}
+import org.scalatest.BeforeAndAfterAll
+
+/**
+ * Test for schema validation during streaming ingestion
+ * Validates streamed schema(source) against existing table(target) schema.
+ */
+
+class CarbonSourceSchemaValidationTest extends Spark2QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll() {
+    sql("DROP TABLE IF EXISTS _carbon_stream_table_")
+  }
+
+  test("Testing validate schema method with correct values ") {
+
+    val spark = SparkSession.builder
+      .appName("StreamIngestSchemaValidation")
+      .master("local")
+      .getOrCreate()
+
+    val carbonSource = new CarbonSource
+    val job = new Job()
+    val warehouseLocation = TestQueryExecutor.warehouse
+
+    sql("CREATE TABLE _carbon_stream_table_(id int,name string)STORED BY 'carbondata'")
+    val tablePath: String = s"$warehouseLocation/default/_carbon_stream_table_"
+    val dataSchema = StructType(Array(StructField("id", IntegerType, true), StructField("name", StringType, true)))
+    val res = carbonSource.prepareWrite(spark, job, Map("path" -> tablePath), dataSchema)
+    assert(res.isInstanceOf[CarbonStreamingOutputWriterFactory])
+  }
+
+  override def afterAll() {
+    sql("DROP TABLE IF EXISTS _carbon_stream_table_")
+  }
+
+}


[02/50] [abbrv] carbondata git commit: [CARBONDATA-1491] Dictionary_exclude columns are not going into no_dictionary flow

Posted by ja...@apache.org.
[CARBONDATA-1491] Dictionary_exclude columns are not going into no_dictionary flow

(1) DICTIONARY_EXCLUDE columns are not considered as no_dictionary columns - while parsing we are not setting nodictionary columns.
(2) For reconstructing defaultValue for newly added no dictionary measure column, logic is changed, as the previous logic can throw IllegalArgumentException for wrong length.
(3) Test cases are added for the same.

This closes #1374


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

Branch: refs/heads/streaming_ingest
Commit: 1e7da59b466ae4f33e3c184db02f218f91a6a2ae
Parents: d4eabbe
Author: dhatchayani <dh...@gmail.com>
Authored: Wed Sep 20 12:03:18 2017 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Thu Sep 21 11:24:39 2017 +0530

----------------------------------------------------------------------
 .../scan/executor/util/RestructureUtil.java     |  7 ++++--
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |  5 +++-
 .../vectorreader/AddColumnTestCases.scala       | 24 ++++++++++++++++++++
 3 files changed, 33 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1e7da59b/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 5e78741..6a281dd 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
@@ -215,13 +215,16 @@ public class RestructureUtil {
    */
   private static Object getNoDictionaryDefaultValue(DataType datatype, byte[] defaultValue) {
     Object noDictionaryDefaultValue = null;
+    String value = null;
     if (!isDefaultValueNull(defaultValue)) {
       switch (datatype) {
         case INT:
-          noDictionaryDefaultValue = ByteUtil.toInt(defaultValue, 0, defaultValue.length);
+          value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+          noDictionaryDefaultValue = Integer.parseInt(value);
           break;
         case LONG:
-          noDictionaryDefaultValue = ByteUtil.toLong(defaultValue, 0, defaultValue.length);
+          value = new String(defaultValue, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+          noDictionaryDefaultValue = Long.parseLong(value);
           break;
         case TIMESTAMP:
           long timestampValue = ByteUtil.toLong(defaultValue, 0, defaultValue.length);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1e7da59b/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 acdec91..42070c4 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
@@ -617,7 +617,10 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
     // by default consider all String cols as dims and if any dictionary include isn't present then
     // add it to noDictionaryDims list. consider all dictionary excludes/include cols as dims
     fields.foreach { field =>
-      if (dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
+      if (dictExcludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
+        noDictionaryDims :+= field.column
+        dimFields += field
+      } else if (dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {
         dimFields += field
       } else if (DataTypeUtil.getDataType(field.dataType.get.toUpperCase) == DataType.TIMESTAMP &&
                  !dictIncludeCols.exists(x => x.equalsIgnoreCase(field.column))) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1e7da59b/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 30485d1..f5ff2e3 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
@@ -418,6 +418,30 @@ class AddColumnTestCases extends Spark2QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS carbon_table")
   }
 
+  test("test to check if intField returns correct result - dictionary exclude") {
+    sqlContext.setConf("carbon.enable.vector.reader", "true")
+    sql("DROP TABLE IF EXISTS carbon_table")
+    sql("CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField TIMESTAMP, decimalField DECIMAL(6,2)) STORED BY 'carbondata'")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(
+      "ALTER TABLE carbon_table ADD COLUMNS(newField INT) TBLPROPERTIES" +
+      "('DEFAULT.VALUE.newField'='67890', 'DICTIONARY_EXCLUDE'='newField')")
+    checkAnswer(sql("SELECT DISTINCT(newField) FROM carbon_table"), Row(67890))
+    sql("DROP TABLE IF EXISTS carbon_table")
+  }
+
+  test("test to check if bigintField returns correct result - dictionary exclude") {
+    sqlContext.setConf("carbon.enable.vector.reader", "true")
+    sql("DROP TABLE IF EXISTS carbon_table")
+    sql("CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField TIMESTAMP, decimalField DECIMAL(6,2)) STORED BY 'carbondata'")
+    sql(s"LOAD DATA LOCAL INPATH '$resourcesPath/restructure/data1.csv' INTO TABLE carbon_table OPTIONS('FILEHEADER'='intField,stringField,charField,timestampField,decimalField')")
+    sql(
+      "ALTER TABLE carbon_table ADD COLUMNS(newField bigint) TBLPROPERTIES" +
+      "('DEFAULT.VALUE.newField'='67890', 'DICTIONARY_EXCLUDE'='newField')")
+    checkAnswer(sql("SELECT DISTINCT(newField) FROM carbon_table"), Row(67890))
+    sql("DROP TABLE IF EXISTS carbon_table")
+  }
+
   test("test to check if shortField returns correct result") {
     sql("DROP TABLE IF EXISTS carbon_table")
     sql("CREATE TABLE carbon_table(intField INT,stringField STRING,charField STRING,timestampField TIMESTAMP, decimalField DECIMAL(6,2)) STORED BY 'carbondata'")


[07/50] [abbrv] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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 24b2981..9f4a8ce 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
@@ -21,11 +21,14 @@ import scala.collection.mutable
 import scala.language.implicitConversions
 
 import org.apache.spark.sql.{DeleteRecords, ShowLoadsCommand, UpdateTable}
-import org.apache.spark.sql.catalyst.{CarbonDDLSqlParser, TableIdentifier}
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.CarbonDDLSqlParser
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit._
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.management.{AlterTableCompactionCommand, CleanFilesCommand, DeleteLoadByIdCommand, DeleteLoadByLoadDateCommand, LoadTableCommand}
+import org.apache.spark.sql.execution.command.partition.{AlterTableDropCarbonPartitionCommand, AlterTableSplitCarbonPartitionCommand}
+import org.apache.spark.sql.execution.command.schema.{AlterTableAddColumnCommand, AlterTableDataTypeChangeCommand, AlterTableDropColumnCommand}
 import org.apache.spark.sql.types.StructField
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -45,10 +48,10 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
       initLexical
       phrase(start)(new lexical.Scanner(input)) match {
         case Success(plan, _) => plan match {
-          case x: LoadTable =>
+          case x: LoadTableCommand =>
             x.inputSqlString = input
             x
-          case x: AlterTableCompaction =>
+          case x: AlterTableCompactionCommand =>
             x.alterTableModel.alterSql = input
             x
           case logicalPlan => logicalPlan
@@ -80,7 +83,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
       case dbName ~ table ~ addInfo =>
         val alterTableAddPartitionModel =
           AlterTableSplitPartitionModel(dbName, table, "0", addInfo)
-        AlterTableSplitPartitionCommand(alterTableAddPartitionModel)
+        AlterTableSplitCarbonPartitionCommand(alterTableAddPartitionModel)
     }
 
   protected lazy val alterSplitPartition: Parser[LogicalPlan] =
@@ -92,7 +95,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         if (partitionId == 0) {
           sys.error("Please use [Alter Table Add Partition] statement to split default partition!")
         }
-        AlterTableSplitPartitionCommand(alterTableSplitPartitionModel)
+        AlterTableSplitCarbonPartitionCommand(alterTableSplitPartitionModel)
     }
 
   protected lazy val alterDropPartition: Parser[LogicalPlan] =
@@ -105,7 +108,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         }
         val alterTableDropPartitionModel =
           AlterTableDropPartitionModel(dbName, table, partitionId, dropWithData)
-        AlterTableDropPartition(alterTableDropPartitionModel)
+        AlterTableDropCarbonPartitionCommand(alterTableDropPartitionModel)
     }
 
 
@@ -115,7 +118,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         val altertablemodel =
           AlterTableModel(convertDbNameToLowerCase(dbName), table, None, compactType,
           Some(System.currentTimeMillis()), null)
-        AlterTableCompaction(altertablemodel)
+        AlterTableCompactionCommand(altertablemodel)
     }
 
   protected lazy val deleteRecords: Parser[LogicalPlan] =
@@ -253,7 +256,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
           validateOptions(optionsList)
         }
         val optionsMap = optionsList.getOrElse(List.empty[(String, String)]).toMap
-        LoadTable(convertDbNameToLowerCase(databaseNameOp), tableName, filePath, Seq(), optionsMap,
+        LoadTableCommand(
+          convertDbNameToLowerCase(databaseNameOp),
+          tableName,
+          filePath,
+          Seq(),
+          optionsMap,
           isOverwrite.isDefined)
     }
 
@@ -262,7 +270,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     (WHERE ~> (SEGMENT ~ "." ~ ID) ~> IN ~> "(" ~> repsep(segmentId, ",")) <~ ")" ~
     opt(";") ^^ {
       case dbName ~ tableName ~ loadids =>
-        DeleteLoadsById(loadids, dbName, tableName.toLowerCase())
+        DeleteLoadByIdCommand(loadids, dbName, tableName.toLowerCase())
     }
 
   protected lazy val deleteLoadsByLoadDate: Parser[LogicalPlan] =
@@ -272,7 +280,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
       case database ~ table ~ condition =>
         condition match {
           case dateField ~ dateValue =>
-            DeleteLoadsByLoadDate(convertDbNameToLowerCase(database),
+            DeleteLoadByLoadDateCommand(convertDbNameToLowerCase(database),
               table.toLowerCase(),
               dateField,
               dateValue)
@@ -282,14 +290,15 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
   protected lazy val cleanFiles: Parser[LogicalPlan] =
     CLEAN ~> FILES ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident <~ opt(";") ^^ {
       case databaseName ~ tableName =>
-        CleanFiles(convertDbNameToLowerCase(databaseName), tableName.toLowerCase())
+        CleanFilesCommand(convertDbNameToLowerCase(databaseName), tableName.toLowerCase())
     }
 
   protected lazy val explainPlan: Parser[LogicalPlan] =
     (EXPLAIN ~> opt(EXTENDED)) ~ startCommand ^^ {
       case isExtended ~ logicalPlan =>
         logicalPlan match {
-          case plan: CreateTable => ExplainCommand(logicalPlan, extended = isExtended.isDefined)
+          case _: CarbonCreateTableCommand =>
+            ExplainCommand(logicalPlan, extended = isExtended.isDefined)
           case _ => ExplainCommand(OneRowRelation)
         }
     }
@@ -317,7 +326,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
             table.toLowerCase,
             columnName.toLowerCase,
             columnNameCopy.toLowerCase)
-        AlterTableDataTypeChange(alterTableChangeDataTypeModel)
+        AlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel)
     }
 
   protected lazy val alterTableAddColumns: Parser[LogicalPlan] =
@@ -386,7 +395,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
           tableModel.dimCols,
           tableModel.msrCols,
           tableModel.highcardinalitydims.getOrElse(Seq.empty))
-        AlterTableAddColumns(alterTableAddColumnsModel)
+        AlterTableAddColumnCommand(alterTableAddColumnsModel)
     }
 
   private def checkFieldDefaultValue(fieldName: String, defaultValueColumnName: String): Boolean = {
@@ -410,7 +419,7 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         val alterTableDropColumnModel = AlterTableDropColumnModel(convertDbNameToLowerCase(dbName),
           table.toLowerCase,
           values.map(_.toLowerCase))
-        AlterTableDropColumns(alterTableDropColumnModel)
+        AlterTableDropColumnCommand(alterTableDropColumnModel)
     }
 
   def getFields(schema: Seq[StructField]): Seq[Field] = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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 b4389a6..52008f2 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
@@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.parser.ParserUtils._
 import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{CreateTableContext, TablePropertyListContext}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.SparkSqlAstBuilder
-import org.apache.spark.sql.execution.command.{BucketFields, CreateTable, Field, PartitionerField, TableModel}
+import org.apache.spark.sql.execution.command.{BucketFields, CarbonCreateTableCommand, Field, PartitionerField, TableModel}
 import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
 
 import org.apache.carbondata.core.util.{CarbonSessionInfo, SessionParams, ThreadLocalSessionInfo}
@@ -146,7 +146,7 @@ class CarbonSqlAstBuilder(conf: SQLConf) extends SparkSqlAstBuilder(conf) {
         tableProperties,
         bucketFields)
 
-      CreateTable(tableModel)
+      CarbonCreateTableCommand(tableModel)
     } else {
       super.visitCreateTable(ctx)
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala b/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
index d00cb84..91121ce 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/Compaction.scala
@@ -17,7 +17,8 @@
 package org.apache.spark.util
 
 import org.apache.spark.sql.{CarbonEnv, SparkSession}
-import org.apache.spark.sql.execution.command.{AlterTableCompaction, AlterTableModel}
+import org.apache.spark.sql.execution.command.AlterTableModel
+import org.apache.spark.sql.execution.command.management.AlterTableCompactionCommand
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.processing.merger.CompactionType
@@ -33,7 +34,7 @@ object Compaction {
     TableAPIUtil.validateTableExists(spark, dbName, tableName)
     if (compactionType.equalsIgnoreCase(CarbonCommonConstants.MAJOR) ||
         compactionType.equalsIgnoreCase(CarbonCommonConstants.MINOR)) {
-      AlterTableCompaction(AlterTableModel(Some(dbName),
+      AlterTableCompactionCommand(AlterTableModel(Some(dbName),
         tableName,
         None,
         compactionType,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
index 72c7426..501402b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/TableLoader.scala
@@ -24,7 +24,7 @@ import scala.collection.{immutable, mutable}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.spark.sql._
-import org.apache.spark.sql.execution.command.LoadTable
+import org.apache.spark.sql.execution.command.management.LoadTableCommand
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.CarbonProperties
@@ -61,7 +61,7 @@ object TableLoader {
 
   def loadTable(spark: SparkSession, dbName: Option[String], tableName: String, inputPaths: String,
       options: scala.collection.immutable.Map[String, String]): Unit = {
-    LoadTable(dbName, tableName, inputPaths, Nil, options, false).run(spark)
+    LoadTableCommand(dbName, tableName, inputPaths, Nil, options, false).run(spark)
   }
 
   def main(args: Array[String]): Unit = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala
index db62eb5..d53065f 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/vectorreader/VectorReaderTestCase.scala
@@ -19,7 +19,7 @@ package org.apache.spark.carbondata.vectorreader
 
 import org.apache.spark.sql.Row
 import org.apache.spark.sql.common.util.Spark2QueryTest
-import org.apache.spark.sql.execution.command.LoadTable
+import org.apache.spark.sql.execution.command.management.LoadTableCommand
 import org.apache.spark.sql.execution.{BatchedDataSourceScanExec, RowDataSourceScanExec}
 import org.scalatest.BeforeAndAfterAll
 


[31/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/DictionaryServerClientDictionary.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/DictionaryServerClientDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/DictionaryServerClientDictionary.java
deleted file mode 100644
index 82605a5..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/DictionaryServerClientDictionary.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.dictionary;
-
-import java.util.Map;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.dictionary.client.DictionaryClient;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
-import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
-
-/**
- * Dictionary implementation along with dictionary server client to get new dictionary values
- */
-public class DictionaryServerClientDictionary implements BiDictionary<Integer, Object> {
-
-  private Dictionary dictionary;
-
-  private DictionaryClient client;
-
-  private Map<Object, Integer> localCache;
-
-  private DictionaryMessage dictionaryMessage;
-
-  private int base;
-
-  public DictionaryServerClientDictionary(Dictionary dictionary, DictionaryClient client,
-      DictionaryMessage key, Map<Object, Integer> localCache) {
-    this.dictionary = dictionary;
-    this.client = client;
-    this.dictionaryMessage = key;
-    this.localCache = localCache;
-    this.base = (dictionary == null ? 0 : dictionary.getDictionaryChunks().getSize() - 1);
-  }
-
-  @Override public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
-    Integer key = getKey(value);
-    if (key == null) {
-      dictionaryMessage.setData(value.toString());
-      DictionaryMessage dictionaryValue = client.getDictionary(dictionaryMessage);
-      key = dictionaryValue.getDictionaryValue();
-      synchronized (localCache) {
-        localCache.put(value, key);
-      }
-      return key + base;
-    }
-    return key;
-  }
-
-  @Override public Integer getKey(Object value) {
-    Integer key = -1;
-    if (dictionary != null) {
-      key = dictionary.getSurrogateKey(value.toString());
-    }
-    if (key == CarbonCommonConstants.INVALID_SURROGATE_KEY) {
-      key = localCache.get(value);
-      if (key != null) {
-        return key + base;
-      }
-    }
-    return key;
-  }
-
-  @Override public Object getValue(Integer key) {
-    throw new UnsupportedOperationException("Not supported here");
-  }
-
-  @Override public int size() {
-    dictionaryMessage.setType(DictionaryMessageType.SIZE);
-    return client.getDictionary(dictionaryMessage).getDictionaryValue() + base;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/DirectDictionary.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/DirectDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/DirectDictionary.java
deleted file mode 100644
index e6cd42f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/DirectDictionary.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.processing.newflow.dictionary;
-
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-
-/**
- * It is used for generating dictionary from value itself, like timestamp can be used directly as
- * dictionary.
- */
-public class DirectDictionary implements BiDictionary<Integer, Object> {
-
-  private DirectDictionaryGenerator dictionaryGenerator;
-
-  public DirectDictionary(DirectDictionaryGenerator dictionaryGenerator) {
-    this.dictionaryGenerator = dictionaryGenerator;
-  }
-
-  @Override
-  public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
-    Integer key = getKey(value);
-    if (key == null) {
-      throw new UnsupportedOperationException("trying to add new entry in DirectDictionary");
-    }
-    return key;
-  }
-
-  @Override
-  public Integer getKey(Object value) {
-    return dictionaryGenerator.generateDirectSurrogateKey(value.toString());
-  }
-
-  @Override
-  public Object getValue(Integer key) {
-    return dictionaryGenerator.getValueFromSurrogate(key);
-  }
-
-  @Override public int size() {
-    return Integer.MAX_VALUE;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/InMemBiDictionary.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/InMemBiDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/InMemBiDictionary.java
deleted file mode 100644
index 7b6d5f1..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/InMemBiDictionary.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.dictionary;
-
-import java.util.Map;
-
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.devapi.DictionaryGenerator;
-import org.apache.carbondata.core.devapi.GeneratingBiDictionary;
-
-import com.google.common.collect.BiMap;
-import com.google.common.collect.HashBiMap;
-
-public class InMemBiDictionary<K, V> extends GeneratingBiDictionary<K, V> {
-
-  private BiMap<K, V> biMap;
-
-  /**
-   * Constructor to create a new dictionary, dictionary key will be generated by specified generator
-   * @param generator
-   */
-  public InMemBiDictionary(DictionaryGenerator generator) {
-    super(generator);
-    biMap = HashBiMap.create();
-  }
-
-  /**
-   * Constructor to create a pre-created dictionary
-   * @param preCreatedDictionary
-   */
-  public InMemBiDictionary(Map<K, V> preCreatedDictionary) {
-    super(new DictionaryGenerator<K, V>() {
-      @Override
-      public K generateKey(V value) throws DictionaryGenerationException {
-        // Since dictionary is provided by preCreated, normally it should not come here
-        throw new DictionaryGenerationException(
-            "encounter new dictionary value in pre-created dictionary:" + value);
-      }
-    });
-    biMap = HashBiMap.create(preCreatedDictionary);
-  }
-
-  @Override
-  public K getKey(V value) {
-    return biMap.inverse().get(value);
-  }
-
-  @Override
-  public V getValue(K key) {
-    return biMap.get(key);
-  }
-
-  @Override
-  protected void put(K key, V value) {
-    // dictionary is immutable, it is append only
-    assert (!biMap.containsKey(key));
-    assert (!biMap.containsValue(value));
-    biMap.put(key, value);
-  }
-
-  @Override
-  public int size() {
-    return biMap.size();
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/PreCreatedDictionary.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/PreCreatedDictionary.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/PreCreatedDictionary.java
deleted file mode 100644
index 19b1cf3..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/dictionary/PreCreatedDictionary.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.dictionary;
-
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-
-public class PreCreatedDictionary implements BiDictionary<Integer, Object> {
-
-  private Dictionary dictionary;
-
-  public PreCreatedDictionary(Dictionary dictionary) {
-    this.dictionary = dictionary;
-  }
-
-  @Override
-  public Integer getOrGenerateKey(Object value) throws DictionaryGenerationException {
-    Integer key = getKey(value);
-    if (key == null) {
-      throw new UnsupportedOperationException("trying to add new entry in PreCreatedDictionary");
-    }
-    return key;
-  }
-
-  @Override
-  public Integer getKey(Object value) {
-    return dictionary.getSurrogateKey(value.toString());
-  }
-
-  @Override
-  public String getValue(Integer key) {
-    return dictionary.getDictionaryValueForKey(key);
-  }
-
-  @Override
-  public int size() {
-    return dictionary.getDictionaryChunks().getSize();
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/BadRecordFoundException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/BadRecordFoundException.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/BadRecordFoundException.java
deleted file mode 100644
index eb95528..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/BadRecordFoundException.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.exception;
-
-public class BadRecordFoundException extends CarbonDataLoadingException {
-  /**
-   * 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 BadRecordFoundException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public BadRecordFoundException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public BadRecordFoundException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/CarbonDataLoadingException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/CarbonDataLoadingException.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/CarbonDataLoadingException.java
deleted file mode 100644
index 6ffdd03..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/CarbonDataLoadingException.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.newflow.exception;
-
-public class CarbonDataLoadingException extends RuntimeException {
-  /**
-   * 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 CarbonDataLoadingException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public CarbonDataLoadingException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public CarbonDataLoadingException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * 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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/NoRetryException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/NoRetryException.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/NoRetryException.java
deleted file mode 100644
index 027b2d0..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/exception/NoRetryException.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.exception;
-
-public class NoRetryException extends RuntimeException {
-
-  /**
-   * 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 NoRetryException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public NoRetryException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public NoRetryException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/CarbonParserFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/CarbonParserFactory.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/CarbonParserFactory.java
deleted file mode 100644
index c37e782..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/CarbonParserFactory.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.parser;
-
-import java.util.List;
-
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.processing.newflow.parser.impl.ArrayParserImpl;
-import org.apache.carbondata.processing.newflow.parser.impl.PrimitiveParserImpl;
-import org.apache.carbondata.processing.newflow.parser.impl.StructParserImpl;
-
-public final class CarbonParserFactory {
-
-  /**
-   * Create parser for the carbon column.
-   *
-   * @param carbonColumn
-   * @param complexDelimiters
-   * @return
-   */
-  public static GenericParser createParser(CarbonColumn carbonColumn, String[] complexDelimiters,
-      String nullFormat) {
-    return createParser(carbonColumn, complexDelimiters, nullFormat, 0);
-  }
-
-  /**
-   * This method may be called recursively if the carbon column is complex type.
-   *
-   * @param carbonColumn
-   * @param complexDelimiters, these delimiters which are used to separate the complex data types.
-   * @param depth              It is like depth of tree, if column has children then depth is 1,
-   *                           And depth becomes 2 if children has children.
-   *                           This depth is used select the complex
-   *                           delimiters
-   * @return GenericParser
-   */
-  private static GenericParser createParser(CarbonColumn carbonColumn, String[] complexDelimiters,
-      String nullFormat, int depth) {
-    switch (carbonColumn.getDataType()) {
-      case ARRAY:
-        List<CarbonDimension> listOfChildDimensions =
-            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
-        // Create array parser with complex delimiter
-        ArrayParserImpl arrayParser = new ArrayParserImpl(complexDelimiters[depth], nullFormat);
-        for (CarbonDimension dimension : listOfChildDimensions) {
-          arrayParser
-              .addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
-        }
-        return arrayParser;
-      case STRUCT:
-        List<CarbonDimension> dimensions =
-            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
-        // Create struct parser with complex delimiter
-        StructParserImpl parser = new StructParserImpl(complexDelimiters[depth], nullFormat);
-        for (CarbonDimension dimension : dimensions) {
-          parser.addChildren(createParser(dimension, complexDelimiters, nullFormat, depth + 1));
-        }
-        return parser;
-      case MAP:
-        throw new UnsupportedOperationException("Complex type Map is not supported yet");
-      default:
-        return new PrimitiveParserImpl();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/ComplexParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/ComplexParser.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/ComplexParser.java
deleted file mode 100644
index 60247a1..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/ComplexParser.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.parser;
-
-/**
- * It parses data string as per complex data type.
- */
-public interface ComplexParser<E> extends GenericParser<E> {
-
-  /**
-   * Children to this parser.
-   * @param parser
-   */
-  void addChildren(GenericParser parser);
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/GenericParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/GenericParser.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/GenericParser.java
deleted file mode 100644
index b745bed..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/GenericParser.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.parser;
-
-/**
- * Parse the data according to implementation, The implementation classes can be struct, array or
- * map datatypes.
- * It remains thread safe as the state of implementation class should not change while
- * calling @{@link GenericParser#parse(Object)} method
- */
-public interface GenericParser<E> {
-
-  /**
-   * Parse the data as per the delimiter
-   * @param data
-   * @return
-   */
-  E parse(Object data);
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/RowParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/RowParser.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/RowParser.java
deleted file mode 100644
index 9795e90..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/RowParser.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.parser;
-
-/**
- * Parse the complete row at once.
- */
-public interface RowParser {
-
-  /**
-   * Parse row.
-   * @param row input row to be parsed.
-   * @return parsed row.
-   */
-  Object[] parseRow(Object[] row);
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/ArrayParserImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/ArrayParserImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/ArrayParserImpl.java
deleted file mode 100644
index 11bbc78..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/ArrayParserImpl.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.parser.impl;
-
-import java.util.regex.Pattern;
-
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.newflow.complexobjects.ArrayObject;
-import org.apache.carbondata.processing.newflow.parser.ComplexParser;
-import org.apache.carbondata.processing.newflow.parser.GenericParser;
-
-import org.apache.commons.lang.ArrayUtils;
-
-/**
- * It parses the string to @{@link ArrayObject} using delimiter.
- * It is thread safe as the state of class don't change while
- * calling @{@link GenericParser#parse(Object)} method
- */
-public class ArrayParserImpl implements ComplexParser<ArrayObject> {
-
-  private Pattern pattern;
-
-  private GenericParser child;
-
-  private String nullFormat;
-
-  public ArrayParserImpl(String delimiter, String nullFormat) {
-    pattern = Pattern.compile(CarbonUtil.delimiterConverter(delimiter));
-    this.nullFormat = nullFormat;
-  }
-
-  @Override
-  public ArrayObject parse(Object data) {
-    if (data != null) {
-      String value = data.toString();
-      if (!value.isEmpty() && !value.equals(nullFormat)) {
-        String[] split = pattern.split(value, -1);
-        if (ArrayUtils.isNotEmpty(split)) {
-          Object[] array = new Object[split.length];
-          for (int i = 0; i < split.length; i++) {
-            array[i] = child.parse(split[i]);
-          }
-          return new ArrayObject(array);
-        }
-      }
-    }
-    return null;
-  }
-
-  @Override
-  public void addChildren(GenericParser parser) {
-    child = parser;
-  }
-}

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/RowParserImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/RowParserImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/RowParserImpl.java
deleted file mode 100644
index 61e4a31..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/RowParserImpl.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.processing.newflow.parser.impl;
-
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
-import org.apache.carbondata.processing.newflow.parser.CarbonParserFactory;
-import org.apache.carbondata.processing.newflow.parser.GenericParser;
-import org.apache.carbondata.processing.newflow.parser.RowParser;
-
-public class RowParserImpl implements RowParser {
-
-  private GenericParser[] genericParsers;
-
-  private int[] outputMapping;
-
-  private int[] inputMapping;
-
-  private int numberOfColumns;
-
-  public RowParserImpl(DataField[] output, CarbonDataLoadConfiguration configuration) {
-    String[] complexDelimiters =
-        (String[]) configuration.getDataLoadProperty(DataLoadProcessorConstants.COMPLEX_DELIMITERS);
-    String nullFormat =
-        configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
-            .toString();
-    DataField[] input = getInput(configuration);
-    genericParsers = new GenericParser[input.length];
-    for (int i = 0; i < genericParsers.length; i++) {
-      genericParsers[i] =
-          CarbonParserFactory.createParser(input[i].getColumn(), complexDelimiters, nullFormat);
-    }
-    outputMapping = new int[output.length];
-    for (int i = 0; i < input.length; i++) {
-      for (int j = 0; j < output.length; j++) {
-        if (input[i].getColumn().equals(output[j].getColumn())) {
-          outputMapping[i] = j;
-          break;
-        }
-      }
-    }
-  }
-
-  public DataField[] getInput(CarbonDataLoadConfiguration configuration) {
-    DataField[] fields = configuration.getDataFields();
-    String[] header = configuration.getHeader();
-    numberOfColumns = header.length;
-    DataField[] input = new DataField[fields.length];
-    inputMapping = new int[input.length];
-    int k = 0;
-    for (int i = 0; i < fields.length; i++) {
-      for (int j = 0; j < numberOfColumns; j++) {
-        if (header[j].equalsIgnoreCase(fields[i].getColumn().getColName())) {
-          input[k] = fields[i];
-          inputMapping[k] = j;
-          k++;
-          break;
-        }
-      }
-    }
-    return input;
-  }
-
-  @Override
-  public Object[] parseRow(Object[] row) {
-    // If number of columns are less in a row then create new array with same size of header.
-    if (row.length < numberOfColumns) {
-      String[] temp = new String[numberOfColumns];
-      System.arraycopy(row, 0, temp, 0, row.length);
-      row = temp;
-    }
-    Object[] out = new Object[genericParsers.length];
-    for (int i = 0; i < genericParsers.length; i++) {
-      Object obj = row[inputMapping[i]];
-      out[outputMapping[i]] = genericParsers[i].parse(obj);
-    }
-    return out;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/StructParserImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/StructParserImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/StructParserImpl.java
deleted file mode 100644
index 3969d9a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/parser/impl/StructParserImpl.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.parser.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.regex.Pattern;
-
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.newflow.complexobjects.StructObject;
-import org.apache.carbondata.processing.newflow.parser.ComplexParser;
-import org.apache.carbondata.processing.newflow.parser.GenericParser;
-
-import org.apache.commons.lang.ArrayUtils;
-
-/**
- * It parses the string to @{@link StructObject} using delimiter.
- * It is thread safe as the state of class don't change while
- * calling @{@link GenericParser#parse(Object)} method
- */
-public class StructParserImpl implements ComplexParser<StructObject> {
-
-  private Pattern pattern;
-
-  private List<GenericParser> children = new ArrayList<>();
-
-  private String nullFormat;
-
-  public StructParserImpl(String delimiter, String nullFormat) {
-    pattern = Pattern.compile(CarbonUtil.delimiterConverter(delimiter));
-    this.nullFormat = nullFormat;
-  }
-
-  @Override
-  public StructObject parse(Object data) {
-    if (data != null) {
-      String value = data.toString();
-      if (!value.isEmpty() && !value.equals(nullFormat)) {
-        String[] split = pattern.split(value, -1);
-        if (ArrayUtils.isNotEmpty(split)) {
-          Object[] array = new Object[children.size()];
-          for (int i = 0; i < split.length && i < array.length; i++) {
-            array[i] = children.get(i).parse(split[i]);
-          }
-          return new StructObject(array);
-        }
-      }
-    }
-    return null;
-  }
-
-  @Override
-  public void addChildren(GenericParser parser) {
-    children.add(parser);
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/partition/Partitioner.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/partition/Partitioner.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/partition/Partitioner.java
deleted file mode 100644
index 85de593..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/partition/Partitioner.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.partition;
-
-/**
- * Partitions the data as per key
- */
-public interface Partitioner<Key> {
-
-  int getPartition(Key key);
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/partition/impl/HashPartitionerImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/partition/impl/HashPartitionerImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/partition/impl/HashPartitionerImpl.java
deleted file mode 100644
index 42f48f0..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/partition/impl/HashPartitionerImpl.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.partition.impl;
-
-import java.util.List;
-
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.processing.newflow.partition.Partitioner;
-
-/**
- * Hash partitioner implementation
- */
-public class HashPartitionerImpl implements Partitioner<Object[]> {
-
-  private int numberOfBuckets;
-
-  private Hash[] hashes;
-
-  public HashPartitionerImpl(List<Integer> indexes, List<ColumnSchema> columnSchemas,
-      int numberOfBuckets) {
-    this.numberOfBuckets = numberOfBuckets;
-    hashes = new Hash[indexes.size()];
-    for (int i = 0; i < indexes.size(); i++) {
-      switch (columnSchemas.get(i).getDataType()) {
-        case SHORT:
-        case INT:
-        case LONG:
-          hashes[i] = new IntegralHash(indexes.get(i));
-          break;
-        case DOUBLE:
-        case FLOAT:
-        case DECIMAL:
-          hashes[i] = new DecimalHash(indexes.get(i));
-          break;
-        default:
-          hashes[i] = new StringHash(indexes.get(i));
-      }
-    }
-  }
-
-  @Override public int getPartition(Object[] objects) {
-    int hashCode = 0;
-    for (Hash hash : hashes) {
-      hashCode += hash.getHash(objects);
-    }
-    return (hashCode & Integer.MAX_VALUE) % numberOfBuckets;
-  }
-
-  private interface Hash {
-    int getHash(Object[] value);
-  }
-
-  private static class IntegralHash implements Hash {
-
-    private int index;
-
-    private IntegralHash(int index) {
-      this.index = index;
-    }
-
-    public int getHash(Object[] value) {
-      return value[index] != null ? Long.valueOf(value[index].toString()).hashCode() : 0;
-    }
-  }
-
-  private static class DecimalHash implements Hash {
-
-    private int index;
-
-    private DecimalHash(int index) {
-      this.index = index;
-    }
-
-    public int getHash(Object[] value) {
-      return value[index] != null ? Double.valueOf(value[index].toString()).hashCode() : 0;
-    }
-  }
-
-  private static class StringHash implements Hash {
-
-    private int index;
-
-    private StringHash(int index) {
-      this.index = index;
-    }
-
-    @Override public int getHash(Object[] value) {
-      return value[index] != null ? value[index].hashCode() : 0;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/row/CarbonRowBatch.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/row/CarbonRowBatch.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/row/CarbonRowBatch.java
deleted file mode 100644
index 1de55e0..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/row/CarbonRowBatch.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.row;
-
-import java.util.NoSuchElementException;
-
-import org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-
-
-/**
- * Batch of rows.
- */
-public class CarbonRowBatch extends CarbonIterator<CarbonRow> {
-
-  private CarbonRow[] rowBatch;
-
-  private int size = 0;
-
-  private int index = 0;
-
-  public CarbonRowBatch(int batchSize) {
-    this.rowBatch = new CarbonRow[batchSize];
-  }
-
-  public void addRow(CarbonRow carbonRow) {
-    rowBatch[size++] = carbonRow;
-  }
-
-  public int getSize() {
-    return size;
-  }
-
-  @Override public boolean hasNext() {
-    return index < size;
-  }
-
-  @Override
-  public CarbonRow next() throws NoSuchElementException {
-    if (hasNext()) {
-      return rowBatch[index++];
-    }
-    throw new NoSuchElementException("no more elements to iterate");
-  }
-
-  @Override public void remove() {
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/row/CarbonSortBatch.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/row/CarbonSortBatch.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/row/CarbonSortBatch.java
deleted file mode 100644
index ba96a96..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/row/CarbonSortBatch.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.row;
-
-import org.apache.carbondata.core.datastore.row.CarbonRow;
-import org.apache.carbondata.processing.newflow.sort.unsafe.merger.UnsafeSingleThreadFinalSortFilesMerger;
-
-/**
- * Batch of sorted rows which are ready to be processed by
- */
-public class CarbonSortBatch extends CarbonRowBatch {
-
-  private UnsafeSingleThreadFinalSortFilesMerger iterator;
-
-  public CarbonSortBatch(UnsafeSingleThreadFinalSortFilesMerger iterator) {
-    super(0);
-    this.iterator = iterator;
-  }
-
-  @Override public boolean hasNext() {
-    return iterator.hasNext();
-  }
-
-  @Override public CarbonRow next() {
-    return new CarbonRow(iterator.next());
-  }
-
-  @Override public void close() {
-    iterator.close();
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/AbstractMergeSorter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/AbstractMergeSorter.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/AbstractMergeSorter.java
deleted file mode 100644
index 5179baa..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/AbstractMergeSorter.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.sort;
-
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.sort.impl.ThreadStatusObserver;
-
-/**
- * The class defines the common methods used in across various type of sort
- */
-public abstract class AbstractMergeSorter implements Sorter {
-  /**
-   * instance of thread status observer
-   */
-  protected ThreadStatusObserver threadStatusObserver;
-
-  /**
-   * Below method will be used to check error in exception
-   */
-  public void checkError() {
-    if (threadStatusObserver.getThrowable() != null) {
-      if (threadStatusObserver.getThrowable() instanceof CarbonDataLoadingException) {
-        throw (CarbonDataLoadingException) threadStatusObserver.getThrowable();
-      } else {
-        throw new CarbonDataLoadingException(threadStatusObserver.getThrowable());
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortScopeOptions.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortScopeOptions.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortScopeOptions.java
deleted file mode 100644
index 2bf8e16..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortScopeOptions.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-/**
- * Sort scope options
- */
-public class SortScopeOptions {
-
-  public static SortScope getSortScope(String sortScope) {
-    if (sortScope == null) {
-      sortScope = CarbonCommonConstants.LOAD_SORT_SCOPE_DEFAULT;
-    }
-    switch (sortScope.toUpperCase()) {
-      case "BATCH_SORT":
-        return SortScope.BATCH_SORT;
-      case "LOCAL_SORT":
-        return SortScope.LOCAL_SORT;
-      case "GLOBAL_SORT":
-        return SortScope.GLOBAL_SORT;
-      case "NO_SORT":
-        return SortScope.NO_SORT;
-      default:
-        return SortScope.LOCAL_SORT;
-    }
-  }
-
-  public static boolean isValidSortOption(String sortScope) {
-    return CarbonUtil.isValidSortOption(sortScope);
-  }
-
-  public enum SortScope {
-    NO_SORT, BATCH_SORT, LOCAL_SORT, GLOBAL_SORT;
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java
deleted file mode 100644
index 62434bc..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SortStepRowUtil.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort;
-
-import org.apache.carbondata.core.util.NonDictionaryUtil;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-
-public class SortStepRowUtil {
-  public static Object[] convertRow(Object[] data, SortParameters parameters) {
-    int measureCount = parameters.getMeasureColCount();
-    int dimensionCount = parameters.getDimColCount();
-    int complexDimensionCount = parameters.getComplexDimColCount();
-    int noDictionaryCount = parameters.getNoDictionaryCount();
-    boolean[] isNoDictionaryDimensionColumn = parameters.getNoDictionaryDimnesionColumn();
-
-    // 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 allCount = 0;
-    int[] dim = new int[dimensionCount];
-    byte[][] nonDicArray = new byte[noDictionaryCount + complexDimensionCount][];
-    Object[] measures = new Object[measureCount];
-    try {
-      // read dimension values
-      for (int i = 0; i < isNoDictionaryDimensionColumn.length; i++) {
-        if (isNoDictionaryDimensionColumn[i]) {
-          nonDicArray[nonDicIndex++] = (byte[]) data[i];
-        } else {
-          dim[index++] = (int) data[allCount];
-        }
-        allCount++;
-      }
-
-      for (int i = 0; i < complexDimensionCount; i++) {
-        nonDicArray[nonDicIndex++] = (byte[]) data[allCount];
-        allCount++;
-      }
-
-      index = 0;
-
-      // read measure values
-      for (int i = 0; i < measureCount; i++) {
-        measures[index++] = data[allCount];
-        allCount++;
-      }
-
-      NonDictionaryUtil.prepareOutObj(holder, dim, nonDicArray, 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/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/Sorter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/Sorter.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/Sorter.java
deleted file mode 100644
index 4a2f5f4..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/Sorter.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.sort;
-
-import java.util.Iterator;
-
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-
-/**
- * This interface sorts all the data of iterators.
- * The life cycle of this interface is initialize -> sort -> close
- */
-public interface Sorter {
-
-  /**
-   * Initialize sorter with sort parameters.
-   *
-   * @param sortParameters
-   */
-  void initialize(SortParameters sortParameters);
-
-  /**
-   * Sorts the data of all iterators, this iterators can be
-   * read parallely depends on implementation.
-   *
-   * @param iterators array of iterators to read data.
-   * @return
-   * @throws CarbonDataLoadingException
-   */
-  Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
-      throws CarbonDataLoadingException;
-
-  /**
-   * Close resources
-   */
-  void close();
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SorterFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SorterFactory.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SorterFactory.java
deleted file mode 100644
index 39a21ad..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/SorterFactory.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.sort;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-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.util.CarbonProperties;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterImpl;
-import org.apache.carbondata.processing.newflow.sort.impl.ParallelReadMergeSorterWithBucketingImpl;
-import org.apache.carbondata.processing.newflow.sort.impl.UnsafeBatchParallelReadMergeSorterImpl;
-import org.apache.carbondata.processing.newflow.sort.impl.UnsafeParallelReadMergeSorterImpl;
-import org.apache.carbondata.processing.newflow.sort.impl.UnsafeParallelReadMergeSorterWithBucketingImpl;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-public class SorterFactory {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SorterFactory.class.getName());
-
-  public static Sorter createSorter(CarbonDataLoadConfiguration configuration, AtomicLong counter) {
-    boolean offheapsort = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
-            CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT));
-    SortScopeOptions.SortScope sortScope = CarbonDataProcessorUtil.getSortScope(configuration);
-    Sorter sorter;
-    if (offheapsort) {
-      if (configuration.getBucketingInfo() != null) {
-        sorter = new UnsafeParallelReadMergeSorterWithBucketingImpl(configuration.getDataFields(),
-            configuration.getBucketingInfo());
-      } else {
-        sorter = new UnsafeParallelReadMergeSorterImpl(counter);
-      }
-    } else {
-      if (configuration.getBucketingInfo() != null) {
-        sorter =
-            new ParallelReadMergeSorterWithBucketingImpl(counter, configuration.getBucketingInfo());
-      } else {
-        sorter = new ParallelReadMergeSorterImpl(counter);
-      }
-    }
-    if (sortScope.equals(SortScopeOptions.SortScope.BATCH_SORT)) {
-      if (configuration.getBucketingInfo() == null) {
-        sorter = new UnsafeBatchParallelReadMergeSorterImpl(counter);
-      } else {
-        LOGGER.warn(
-            "Batch sort is not enabled in case of bucketing. Falling back to " + sorter.getClass()
-                .getName());
-      }
-    }
-    return sorter;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java
deleted file mode 100644
index 5a8a2c8..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterImpl.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.newflow.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.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortIntermediateFileMerger;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-import org.apache.carbondata.processing.store.SingleThreadFinalSortFilesMerger;
-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.
- */
-public class ParallelReadMergeSorterImpl extends AbstractMergeSorter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ParallelReadMergeSorterImpl.class.getName());
-
-  private SortParameters sortParameters;
-
-  private SortIntermediateFileMerger intermediateFileMerger;
-
-  private SingleThreadFinalSortFilesMerger finalMerger;
-
-  private AtomicLong rowCounter;
-
-  public ParallelReadMergeSorterImpl(AtomicLong rowCounter) {
-    this.rowCounter = rowCounter;
-  }
-
-  @Override
-  public void initialize(SortParameters sortParameters) {
-    this.sortParameters = sortParameters;
-    intermediateFileMerger = new SortIntermediateFileMerger(sortParameters);
-    String[] storeLocations =
-        CarbonDataProcessorUtil.getLocalDataFolderLocation(
-            sortParameters.getDatabaseName(), sortParameters.getTableName(),
-            String.valueOf(sortParameters.getTaskNo()), sortParameters.getPartitionID(),
-            sortParameters.getSegmentId() + "", false, false);
-    // Set the data file location
-    String[] dataFolderLocations = CarbonDataProcessorUtil.arrayAppend(storeLocations,
-        File.separator, CarbonCommonConstants.SORT_TEMP_FILE_LOCATION);
-    finalMerger =
-        new SingleThreadFinalSortFilesMerger(dataFolderLocations, sortParameters.getTableName(),
-            sortParameters.getDimColCount(),
-            sortParameters.getComplexDimColCount(), sortParameters.getMeasureColCount(),
-            sortParameters.getNoDictionaryCount(), sortParameters.getMeasureDataType(),
-            sortParameters.getNoDictionaryDimnesionColumn(),
-            sortParameters.getNoDictionarySortColumn());
-  }
-
-  @Override
-  public Iterator<CarbonRowBatch>[] sort(Iterator<CarbonRowBatch>[] iterators)
-      throws CarbonDataLoadingException {
-    SortDataRows sortDataRow = new SortDataRows(sortParameters, intermediateFileMerger);
-    final int batchSize = CarbonProperties.getInstance().getBatchSize();
-    try {
-      sortDataRow.initialize();
-    } catch (CarbonSortKeyAndGroupByException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
-    this.threadStatusObserver = new ThreadStatusObserver(executorService);
-
-    try {
-      for (int i = 0; i < iterators.length; i++) {
-        executorService.execute(
-            new SortIteratorThread(iterators[i], sortDataRow, batchSize, rowCounter,
-                threadStatusObserver));
-      }
-      executorService.shutdown();
-      executorService.awaitTermination(2, TimeUnit.DAYS);
-      processRowToNextStep(sortDataRow, sortParameters);
-    } catch (Exception e) {
-      checkError();
-      throw new CarbonDataLoadingException("Problem while shutdown the server ", e);
-    }
-    checkError();
-    try {
-      intermediateFileMerger.finish();
-      intermediateFileMerger = null;
-      finalMerger.startFinalMerge();
-    } catch (CarbonDataWriterException e) {
-      throw new CarbonDataLoadingException(e);
-    } catch (CarbonSortKeyAndGroupByException e) {
-      throw new CarbonDataLoadingException(e);
-    }
-
-    // Creates the iterator to read from merge sorter.
-    Iterator<CarbonRowBatch> batchIterator = new CarbonIterator<CarbonRowBatch>() {
-
-      @Override
-      public boolean hasNext() {
-        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;
-      }
-    };
-    return new Iterator[] { batchIterator };
-  }
-
-  @Override public void close() {
-    if (intermediateFileMerger != null) {
-      intermediateFileMerger.close();
-    }
-  }
-
-  /**
-   * Below method will be used to process data to next step
-   */
-  private boolean processRowToNextStep(SortDataRows sortDataRows, SortParameters parameters)
-      throws CarbonDataLoadingException {
-    if (null == sortDataRows) {
-      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 {
-      // start sorting
-      sortDataRows.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);
-    }
-  }
-
-  /**
-   * 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 Object[][] buffer;
-
-    private AtomicLong rowCounter;
-
-    private ThreadStatusObserver observer;
-
-    public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortDataRows sortDataRows,
-        int batchSize, AtomicLong rowCounter, ThreadStatusObserver observer) {
-      this.iterator = iterator;
-      this.sortDataRows = sortDataRows;
-      this.buffer = new Object[batchSize][];
-      this.rowCounter = rowCounter;
-      this.observer = 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) {
-              buffer[i++] = row.getData();
-            }
-          }
-          if (i > 0) {
-            sortDataRows.addRowBatch(buffer, i);
-            rowCounter.getAndAdd(i);
-          }
-        }
-      } catch (Exception e) {
-        LOGGER.error(e);
-        observer.notifyFailed(e);
-      }
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
deleted file mode 100644
index 7e013e0..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/sort/impl/ParallelReadMergeSorterWithBucketingImpl.java
+++ /dev/null
@@ -1,276 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.newflow.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.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-import org.apache.carbondata.processing.newflow.sort.AbstractMergeSorter;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortIntermediateFileMerger;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
-import org.apache.carbondata.processing.store.SingleThreadFinalSortFilesMerger;
-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()), bucketId,
-            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.getDimColCount(), sortParameters.getComplexDimColCount(),
-            sortParameters.getMeasureColCount(), sortParameters.getNoDictionaryCount(),
-            sortParameters.getMeasureDataType(), sortParameters.getNoDictionaryDimnesionColumn(),
-            this.sortParameters.getNoDictionarySortColumn());
-  }
-
-  @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.getPartitionID(), 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;
-    }
-  }
-}


[34/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..7f98d72
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateFileMerger.java
@@ -0,0 +1,367 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.merger;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.AbstractQueue;
+import java.util.Arrays;
+import java.util.PriorityQueue;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.util.CarbonUtil;
+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.TempSortFileWriter;
+import org.apache.carbondata.processing.sort.sortdata.TempSortFileWriterFactory;
+
+public class UnsafeIntermediateFileMerger implements Runnable {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeIntermediateFileMerger.class.getName());
+
+  /**
+   * recordHolderHeap
+   */
+  private AbstractQueue<SortTempChunkHolder> recordHolderHeap;
+
+  /**
+   * fileCounter
+   */
+  private int fileCounter;
+
+  /**
+   * stream
+   */
+  private DataOutputStream stream;
+
+  /**
+   * totalNumberOfRecords
+   */
+  private int totalNumberOfRecords;
+
+  /**
+   * writer
+   */
+  private TempSortFileWriter writer;
+
+  private SortParameters mergerParameters;
+
+  private File[] intermediateFiles;
+
+  private File outPutFile;
+
+  private boolean[] noDictionarycolumnMapping;
+
+  private long[] nullSetWords;
+
+  private ByteBuffer rowData;
+
+  /**
+   * IntermediateFileMerger Constructor
+   */
+  public UnsafeIntermediateFileMerger(SortParameters mergerParameters, File[] intermediateFiles,
+      File outPutFile) {
+    this.mergerParameters = mergerParameters;
+    this.fileCounter = intermediateFiles.length;
+    this.intermediateFiles = intermediateFiles;
+    this.outPutFile = outPutFile;
+    noDictionarycolumnMapping = mergerParameters.getNoDictionaryDimnesionColumn();
+    this.nullSetWords = new long[((mergerParameters.getMeasureColCount() - 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 run() {
+    long intermediateMergeStartTime = System.currentTimeMillis();
+    int fileConterConst = fileCounter;
+    boolean isFailed = false;
+    try {
+      startSorting();
+      initialize();
+      while (hasNext()) {
+        writeDataTofile(next());
+      }
+      double intermediateMergeCostTime =
+          (System.currentTimeMillis() - intermediateMergeStartTime) / 1000.0;
+      LOGGER.info("============================== Intermediate Merge of " + fileConterConst
+          + " Sort Temp Files Cost Time: " + intermediateMergeCostTime + "(s)");
+    } catch (Exception e) {
+      LOGGER.error(e, "Problem while intermediate merging");
+      isFailed = true;
+    } finally {
+      CarbonUtil.closeStreams(this.stream);
+      if (null != writer) {
+        writer.finish();
+      }
+      if (!isFailed) {
+        try {
+          finish();
+        } catch (CarbonSortKeyAndGroupByException e) {
+          LOGGER.error(e, "Problem while deleting the merge file");
+        }
+      } else {
+        if (outPutFile.delete()) {
+          LOGGER.error("Problem while deleting the merge file");
+        }
+      }
+    }
+  }
+
+  /**
+   * This method is responsible for initializing the out stream
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private void initialize() throws CarbonSortKeyAndGroupByException {
+    if (!mergerParameters.isSortFileCompressionEnabled() && !mergerParameters.isPrefetch()) {
+      try {
+        this.stream = new DataOutputStream(
+            new BufferedOutputStream(new FileOutputStream(outPutFile),
+                mergerParameters.getFileWriteBufferSize()));
+        this.stream.writeInt(this.totalNumberOfRecords);
+      } catch (FileNotFoundException e) {
+        throw new CarbonSortKeyAndGroupByException("Problem while getting the file", e);
+      } catch (IOException e) {
+        throw new CarbonSortKeyAndGroupByException("Problem while writing the data to file", e);
+      }
+    } else {
+      writer = TempSortFileWriterFactory.getInstance()
+          .getTempSortFileWriter(mergerParameters.isSortFileCompressionEnabled(),
+              mergerParameters.getDimColCount(), mergerParameters.getComplexDimColCount(),
+              mergerParameters.getMeasureColCount(), mergerParameters.getNoDictionaryCount(),
+              mergerParameters.getFileWriteBufferSize());
+      writer.initiaize(outPutFile, totalNumberOfRecords);
+    }
+  }
+
+  /**
+   * This method will be used to get the sorted record from file
+   *
+   * @return sorted record sorted record
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  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
+    // be based on comparator we are passing the heap
+    // when will call poll it will always delete root of the tree and then
+    // it does trickel down operation complexity is log(n)
+    SortTempChunkHolder poll = this.recordHolderHeap.poll();
+
+    // get the row from chunk
+    row = poll.getRow();
+
+    // check if there no entry present
+    if (!poll.hasNext()) {
+      // if chunk is empty then close the stream
+      poll.close();
+
+      // change the file counter
+      --this.fileCounter;
+
+      // reaturn row
+      return row;
+    }
+
+    // read new row
+    poll.readRow();
+
+    // add to heap
+    this.recordHolderHeap.add(poll);
+
+    // return row
+    return row;
+  }
+
+  /**
+   * Below method will be used to start storing process This method will get
+   * all the temp files present in sort temp folder then it will create the
+   * record holder heap and then it will read first record from each file and
+   * initialize the heap
+   *
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private void startSorting() throws CarbonSortKeyAndGroupByException {
+    LOGGER.info("Number of temp file: " + this.fileCounter);
+
+    // create record holder heap
+    createRecordHolderQueue(intermediateFiles);
+
+    // iterate over file list and create chunk holder and add to heap
+    LOGGER.info("Started adding first record from each file");
+
+    SortTempChunkHolder sortTempFileChunkHolder = null;
+
+    for (File tempFile : intermediateFiles) {
+      // create chunk holder
+      sortTempFileChunkHolder = new UnsafeSortTempFileChunkHolder(tempFile, mergerParameters);
+
+      sortTempFileChunkHolder.readRow();
+      this.totalNumberOfRecords += sortTempFileChunkHolder.numberOfRows();
+
+      // add to heap
+      this.recordHolderHeap.add(sortTempFileChunkHolder);
+    }
+
+    LOGGER.info("Heap Size" + this.recordHolderHeap.size());
+  }
+
+  /**
+   * This method will be used to create the heap which will be used to hold
+   * the chunk of data
+   *
+   * @param listFiles list of temp files
+   */
+  private void createRecordHolderQueue(File[] listFiles) {
+    // creating record holder heap
+    this.recordHolderHeap = new PriorityQueue<SortTempChunkHolder>(listFiles.length);
+  }
+
+  /**
+   * This method will be used to get the sorted row
+   *
+   * @return sorted row
+   * @throws CarbonSortKeyAndGroupByException
+   */
+  private Object[] next() throws CarbonSortKeyAndGroupByException {
+    return getSortedRecordFromFile();
+  }
+
+  /**
+   * This method will be used to check whether any more element is present or
+   * not
+   *
+   * @return more element is present
+   */
+  private boolean hasNext() {
+    return this.fileCounter > 0;
+  }
+
+  /**
+   * Below method will be used to write data to file
+   *
+   * @throws CarbonSortKeyAndGroupByException problem while writing
+   */
+  private void writeDataTofile(Object[] row) throws CarbonSortKeyAndGroupByException, IOException {
+    int dimCount = 0;
+    int size = 0;
+    DataType[] type = mergerParameters.getMeasureDataType();
+    for (; dimCount < noDictionarycolumnMapping.length; dimCount++) {
+      if (noDictionarycolumnMapping[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 =
+        mergerParameters.getDimColCount() + mergerParameters.getComplexDimColCount();
+    int measureSize = mergerParameters.getMeasureColCount();
+    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 < measureSize; mesCount++) {
+      Object value = row[mesCount + dimensionSize];
+      if (null != value) {
+        switch (type[mesCount]) {
+          case SHORT:
+            rowData.putShort(size, (Short) value);
+            size += 2;
+            break;
+          case INT:
+            rowData.putInt(size, (Integer) value);
+            size += 4;
+            break;
+          case LONG:
+            rowData.putLong(size, (Long) value);
+            size += 8;
+            break;
+          case DOUBLE:
+            rowData.putDouble(size, (Double) value);
+            size += 8;
+            break;
+          case DECIMAL:
+            byte[] bigDecimalInBytes = (byte[]) value;
+            rowData.putShort(size, (short)bigDecimalInBytes.length);
+            size += 2;
+            for (int i = 0; i < bigDecimalInBytes.length; i++) {
+              rowData.put(size++, bigDecimalInBytes[i]);
+            }
+            break;
+        }
+        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 {
+    if (recordHolderHeap != null) {
+      int size = recordHolderHeap.size();
+      for (int i = 0; i < size; i++) {
+        recordHolderHeap.poll().close();
+      }
+    }
+    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/349c59c7/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
new file mode 100644
index 0000000..c774d8f
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeIntermediateMerger.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.merger;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+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.util.CarbonProperties;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+
+/**
+ * It does mergesort intermediate files to big file.
+ */
+public class UnsafeIntermediateMerger {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeIntermediateMerger.class.getName());
+
+  /**
+   * executorService
+   */
+  private ExecutorService executorService;
+  /**
+   * rowPages
+   */
+  private List<UnsafeCarbonRowPage> rowPages;
+
+  private List<UnsafeInMemoryIntermediateDataMerger> mergedPages;
+
+  private SortParameters parameters;
+
+  private final Object lockObject = new Object();
+
+  private boolean offHeap;
+
+  private List<File> procFiles;
+
+  public UnsafeIntermediateMerger(SortParameters parameters) {
+    this.parameters = parameters;
+    // processed file list
+    this.rowPages = new ArrayList<UnsafeCarbonRowPage>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    this.mergedPages = new ArrayList<>();
+    this.executorService = Executors.newFixedThreadPool(parameters.getNumberOfCores());
+    this.offHeap = Boolean.parseBoolean(CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT,
+            CarbonCommonConstants.ENABLE_OFFHEAP_SORT_DEFAULT));
+    this.procFiles = new ArrayList<File>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+  }
+
+  public void addDataChunkToMerge(UnsafeCarbonRowPage rowPage) {
+    // add sort temp filename to and arrayList. When the list size reaches 20 then
+    // intermediate merging of sort temp files will be triggered
+    synchronized (lockObject) {
+      rowPages.add(rowPage);
+    }
+  }
+
+  public void addFileToMerge(File sortTempFile) {
+    // add sort temp filename to and arrayList. When the list size reaches 20 then
+    // intermediate merging of sort temp files will be triggered
+    synchronized (lockObject) {
+      procFiles.add(sortTempFile);
+    }
+  }
+
+  public void startFileMergingIfPossible() {
+    File[] fileList;
+    if (procFiles.size() >= parameters.getNumberOfIntermediateFileToBeMerged()) {
+      synchronized (lockObject) {
+        fileList = procFiles.toArray(new File[procFiles.size()]);
+        this.procFiles = new ArrayList<File>();
+      }
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("Sumitting request for intermediate merging no of files: " + fileList.length);
+      }
+      startIntermediateMerging(fileList);
+    }
+  }
+
+  /**
+   * Below method will be used to start the intermediate file merging
+   *
+   * @param intermediateFiles
+   */
+  private void startIntermediateMerging(File[] intermediateFiles) {
+    //pick a temp location randomly
+    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);
+    UnsafeIntermediateFileMerger merger =
+        new UnsafeIntermediateFileMerger(parameters, intermediateFiles, file);
+    executorService.execute(merger);
+  }
+
+  public void startInmemoryMergingIfPossible() throws CarbonSortKeyAndGroupByException {
+    UnsafeCarbonRowPage[] localRowPages;
+    if (rowPages.size() >= parameters.getNumberOfIntermediateFileToBeMerged()) {
+      int totalRows = 0;
+      synchronized (lockObject) {
+        totalRows = getTotalNumberOfRows(rowPages);
+        if (totalRows <= 0) {
+          return;
+        }
+        localRowPages = rowPages.toArray(new UnsafeCarbonRowPage[rowPages.size()]);
+        this.rowPages = new ArrayList<>();
+      }
+      if (LOGGER.isDebugEnabled()) {
+        LOGGER.debug("Sumitting request for intermediate merging of in-memory pages : "
+            + localRowPages.length);
+      }
+      startIntermediateMerging(localRowPages, totalRows);
+    }
+  }
+
+  /**
+   * Below method will be used to start the intermediate file merging
+   *
+   * @param rowPages
+   */
+  private void startIntermediateMerging(UnsafeCarbonRowPage[] rowPages, int totalRows)
+      throws CarbonSortKeyAndGroupByException {
+    UnsafeInMemoryIntermediateDataMerger merger =
+        new UnsafeInMemoryIntermediateDataMerger(rowPages, totalRows);
+    mergedPages.add(merger);
+    executorService.execute(merger);
+  }
+
+  private int getTotalNumberOfRows(List<UnsafeCarbonRowPage> unsafeCarbonRowPages) {
+    int totalSize = 0;
+    for (UnsafeCarbonRowPage unsafeCarbonRowPage : unsafeCarbonRowPages) {
+      totalSize += unsafeCarbonRowPage.getBuffer().getActualSize();
+    }
+    return totalSize;
+  }
+
+  public void finish() throws CarbonSortKeyAndGroupByException {
+    try {
+      executorService.shutdown();
+      executorService.awaitTermination(2, TimeUnit.DAYS);
+    } catch (InterruptedException e) {
+      throw new CarbonSortKeyAndGroupByException("Problem while shutdown the server ", e);
+    }
+  }
+
+  public void close() {
+    if (executorService.isShutdown()) {
+      executorService.shutdownNow();
+    }
+    rowPages.clear();
+    rowPages = null;
+  }
+
+  public List<UnsafeCarbonRowPage> getRowPages() {
+    return rowPages;
+  }
+
+  public List<UnsafeInMemoryIntermediateDataMerger> getMergedPages() {
+    return mergedPages;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..32b31d7
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/merger/UnsafeSingleThreadFinalSortFilesMerger.java
@@ -0,0 +1,259 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.merger;
+
+import java.io.File;
+import java.io.FileFilter;
+import java.util.AbstractQueue;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.PriorityQueue;
+
+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.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.SortTempChunkHolder;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeFinalMergePageHolder;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeInmemoryHolder;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeSortTempFileChunkHolder;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+
+public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(UnsafeSingleThreadFinalSortFilesMerger.class.getName());
+
+  /**
+   * fileCounter
+   */
+  private int fileCounter;
+
+  /**
+   * recordHolderHeap
+   */
+  private AbstractQueue<SortTempChunkHolder> recordHolderHeapLocal;
+
+  private SortParameters parameters;
+
+  /**
+   * tempFileLocation
+   */
+  private String[] tempFileLocation;
+
+  private String tableName;
+
+  private boolean isStopProcess;
+
+  public UnsafeSingleThreadFinalSortFilesMerger(SortParameters parameters,
+      String[] tempFileLocation) {
+    this.parameters = parameters;
+    this.tempFileLocation = tempFileLocation;
+    this.tableName = parameters.getTableName();
+  }
+
+  /**
+   * This method will be used to merger the merged files
+   *
+   */
+  public void startFinalMerge(UnsafeCarbonRowPage[] rowPages,
+      List<UnsafeInMemoryIntermediateDataMerger> merges) throws CarbonDataWriterException {
+    startSorting(rowPages, merges);
+  }
+
+  /**
+   * Below method will be used to start storing process This method will get
+   * all the temp files present in sort temp folder then it will create the
+   * record holder heap and then it will read first record from each file and
+   * initialize the heap
+   *
+   */
+  private void startSorting(UnsafeCarbonRowPage[] rowPages,
+      List<UnsafeInMemoryIntermediateDataMerger> merges) throws CarbonDataWriterException {
+    try {
+      List<File> filesToMergeSort = getFilesToMergeSort();
+      this.fileCounter = rowPages.length + filesToMergeSort.size() + merges.size();
+      if (fileCounter == 0) {
+        LOGGER.info("No files to merge sort");
+        return;
+      }
+      LOGGER.info("Number of row pages: " + this.fileCounter);
+
+      // create record holder heap
+      createRecordHolderQueue();
+
+      // iterate over file list and create chunk holder and add to heap
+      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());
+
+        // initialize
+        sortTempFileChunkHolder.readRow();
+
+        recordHolderHeapLocal.add(sortTempFileChunkHolder);
+      }
+
+      for (final UnsafeInMemoryIntermediateDataMerger merger : merges) {
+
+        SortTempChunkHolder sortTempFileChunkHolder =
+            new UnsafeFinalMergePageHolder(merger, parameters.getNoDictionarySortColumn(),
+                parameters.getDimColCount() + parameters.getComplexDimColCount() + parameters
+                    .getMeasureColCount());
+
+        // initialize
+        sortTempFileChunkHolder.readRow();
+
+        recordHolderHeapLocal.add(sortTempFileChunkHolder);
+      }
+
+      for (final File file : filesToMergeSort) {
+
+        SortTempChunkHolder sortTempFileChunkHolder =
+            new UnsafeSortTempFileChunkHolder(file, parameters);
+
+        // initialize
+        sortTempFileChunkHolder.readRow();
+
+        recordHolderHeapLocal.add(sortTempFileChunkHolder);
+      }
+
+      LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
+    } catch (Exception e) {
+      LOGGER.error(e);
+      throw new CarbonDataWriterException(e.getMessage());
+    }
+  }
+
+  private List<File> getFilesToMergeSort() {
+    FileFilter fileFilter = new FileFilter() {
+      public boolean accept(File pathname) {
+        return pathname.getName().startsWith(tableName);
+      }
+    };
+
+    // get all the merged files
+    List<File> files = new ArrayList<File>(tempFileLocation.length);
+    for (String tempLoc : tempFileLocation)
+    {
+      File[] subFiles = new File(tempLoc).listFiles(fileFilter);
+      if (null != subFiles && subFiles.length > 0)
+      {
+        files.addAll(Arrays.asList(subFiles));
+      }
+    }
+
+    return files;
+  }
+
+  /**
+   * This method will be used to create the heap which will be used to hold
+   * the chunk of data
+   */
+  private void createRecordHolderQueue() {
+    // creating record holder heap
+    this.recordHolderHeapLocal = new PriorityQueue<SortTempChunkHolder>(fileCounter);
+  }
+
+  /**
+   * This method will be used to get the sorted row
+   *
+   * @return sorted row
+   */
+  public Object[] next() {
+    return SortStepRowUtil.convertRow(getSortedRecordFromFile(), parameters);
+  }
+
+  /**
+   * This method will be used to get the sorted record from file
+   *
+   * @return sorted record sorted record
+   */
+  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
+    // be based on comparator we are passing the heap
+    // when will call poll it will always delete root of the tree and then
+    // it does trickel down operation complexity is log(n)
+    SortTempChunkHolder poll = this.recordHolderHeapLocal.poll();
+
+    // get the row from chunk
+    row = poll.getRow();
+
+    // check if there no entry present
+    if (!poll.hasNext()) {
+      // if chunk is empty then close the stream
+      poll.close();
+
+      // change the file counter
+      --this.fileCounter;
+
+      // reaturn row
+      return row;
+    }
+
+    // read new row
+    try {
+      poll.readRow();
+    } catch (Exception e) {
+      throw new CarbonDataWriterException(e.getMessage(), e);
+    }
+
+    // add to heap
+    this.recordHolderHeapLocal.add(poll);
+
+    // return row
+    return row;
+  }
+
+  /**
+   * This method will be used to check whether any more element is present or
+   * not
+   *
+   * @return more element is present
+   */
+  public boolean hasNext() {
+    return this.fileCounter > 0;
+  }
+
+  public void clear() {
+    if (null != recordHolderHeapLocal) {
+      for (SortTempChunkHolder pageHolder : recordHolderHeapLocal) {
+        pageHolder.close();
+      }
+      recordHolderHeapLocal = null;
+    }
+  }
+
+  public boolean isStopProcess() {
+    return isStopProcess;
+  }
+
+  public void setStopProcess(boolean stopProcess) {
+    isStopProcess = stopProcess;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/sort/TimSort.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/sort/TimSort.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/sort/TimSort.java
new file mode 100644
index 0000000..dac3b47
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/sort/TimSort.java
@@ -0,0 +1,986 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.sort;
+
+import java.util.Comparator;
+
+import org.apache.spark.util.collection.SortDataFormat;
+
+/**
+ * A port of the Apache Spark's TimSort and they originally ported from Android TimSort class,
+ * which utilizes a "stable, adaptive, iterative mergesort."
+ * See the method comment on sort() for more details.
+ *
+ * This has been kept in Java with the original style in order to match very closely with the
+ * Android source code, and thus be easy to verify correctness. The class is package private. We put
+ * a simple Scala wrapper {@link org.apache.spark.util.collection.Sorter}, which is available to
+ * package org.apache.spark.
+ *
+ * The purpose of the port is to generalize the interface to the sort to accept input data formats
+ * besides simple arrays where every element is sorted individually. For instance, the AppendOnlyMap
+ * uses this to sort an Array with alternating elements of the form [key, value, key, value].
+ * This generalization comes with minimal overhead -- see SortDataFormat for more information.
+ *
+ * We allow key reuse to prevent creating many key objects -- see SortDataFormat.
+ *
+ * @see SortDataFormat
+ * @see org.apache.spark.util.collection.Sorter
+ */
+public class TimSort<K, Buffer> {
+
+  /**
+   * This is the minimum sized sequence that will be merged.  Shorter
+   * sequences will be lengthened by calling binarySort.  If the entire
+   * array is less than this length, no merges will be performed.
+   *
+   * This constant should be a power of two.  It was 64 in Tim Peter's C
+   * implementation, but 32 was empirically determined to work better in
+   * this implementation.  In the unlikely event that you set this constant
+   * to be a number that's not a power of two, you'll need to change the
+   * minRunLength computation.
+   *
+   * If you decrease this constant, you must change the stackLen
+   * computation in the TimSort constructor, or you risk an
+   * ArrayOutOfBounds exception.  See listsort.txt for a discussion
+   * of the minimum stack length required as a function of the length
+   * of the array being sorted and the minimum merge sequence length.
+   */
+  private static final int MIN_MERGE = 32;
+
+  private final SortDataFormat<K, Buffer> s;
+
+  public TimSort(SortDataFormat<K, Buffer> sortDataFormat) {
+    this.s = sortDataFormat;
+  }
+
+  /**
+   * A stable, adaptive, iterative mergesort that requires far fewer than
+   * n lg(n) comparisons when running on partially sorted arrays, while
+   * offering performance comparable to a traditional mergesort when run
+   * on random arrays.  Like all proper mergesorts, this sort is stable and
+   * runs O(n log n) time (worst case).  In the worst case, this sort requires
+   * temporary storage space for n/2 object references; in the best case,
+   * it requires only a small constant amount of space.
+   *
+   * This implementation was adapted from Tim Peters's list sort for
+   * Python, which is described in detail here:
+   *
+   *   http://svn.python.org/projects/python/trunk/Objects/listsort.txt
+   *
+   * Tim's C code may be found here:
+   *
+   *   http://svn.python.org/projects/python/trunk/Objects/listobject.c
+   *
+   * The underlying techniques are described in this paper (and may have
+   * even earlier origins):
+   *
+   *  "Optimistic Sorting and Information Theoretic Complexity"
+   *  Peter McIlroy
+   *  SODA (Fourth Annual ACM-SIAM Symposium on Discrete Algorithms),
+   *  pp 467-474, Austin, Texas, 25-27 January 1993.
+   *
+   * While the API to this class consists solely of static methods, it is
+   * (privately) instantiable; a TimSort instance holds the state of an ongoing
+   * sort, assuming the input array is large enough to warrant the full-blown
+   * TimSort. Small arrays are sorted in place, using a binary insertion sort.
+   */
+  public void sort(Buffer a, int lo, int hi, Comparator<? super K> c) {
+    assert c != null;
+
+    int nRemaining  = hi - lo;
+    if (nRemaining < 2) {
+      return;  // Arrays of size 0 and 1 are always sorted
+    }
+
+    // If array is small, do a "mini-TimSort" with no merges
+    if (nRemaining < MIN_MERGE) {
+      int initRunLen = countRunAndMakeAscending(a, lo, hi, c);
+      binarySort(a, lo, hi, lo + initRunLen, c);
+      return;
+    }
+
+    /*
+     * March over the array once, left to right, finding natural runs,
+     * extending short natural runs to minRun elements, and merging runs
+     * to maintain stack invariant.
+     */
+    SortState sortState = new SortState(a, c, hi - lo);
+    int minRun = minRunLength(nRemaining);
+    do {
+      // Identify next run
+      int runLen = countRunAndMakeAscending(a, lo, hi, c);
+
+      // If run is short, extend to min(minRun, nRemaining)
+      if (runLen < minRun) {
+        int force = nRemaining <= minRun ? nRemaining : minRun;
+        binarySort(a, lo, lo + force, lo + runLen, c);
+        runLen = force;
+      }
+
+      // Push run onto pending-run stack, and maybe merge
+      sortState.pushRun(lo, runLen);
+      sortState.mergeCollapse();
+
+      // Advance to find next run
+      lo += runLen;
+      nRemaining -= runLen;
+    } while (nRemaining != 0);
+
+    // Merge all remaining runs to complete sort
+    assert lo == hi;
+    sortState.mergeForceCollapse();
+    assert sortState.stackSize == 1;
+  }
+
+  /**
+   * Sorts the specified portion of the specified array using a binary
+   * insertion sort.  This is the best method for sorting small numbers
+   * of elements.  It requires O(n log n) compares, but O(n^2) data
+   * movement (worst case).
+   *
+   * If the initial part of the specified range is already sorted,
+   * this method can take advantage of it: the method assumes that the
+   * elements from index {@code lo}, inclusive, to {@code start},
+   * exclusive are already sorted.
+   *
+   * @param a the array in which a range is to be sorted
+   * @param lo the index of the first element in the range to be sorted
+   * @param hi the index after the last element in the range to be sorted
+   * @param start the index of the first element in the range that is
+   *        not already known to be sorted ({@code lo <= start <= hi})
+   * @param c comparator to used for the sort
+   */
+  @SuppressWarnings("fallthrough")
+  private void binarySort(Buffer a, int lo, int hi, int start, Comparator<? super K> c) {
+    assert lo <= start && start <= hi;
+    if (start == lo) {
+      start++;
+    }
+
+    K key0 = s.newKey();
+    K key1 = s.newKey();
+
+    Buffer pivotStore = s.allocate(1);
+    for ( ; start < hi; start++) {
+      s.copyElement(a, start, pivotStore, 0);
+      K pivot = s.getKey(pivotStore, 0, key0);
+
+      // Set left (and right) to the index where a[start] (pivot) belongs
+      int left = lo;
+      int right = start;
+      assert left <= right;
+      /*
+       * Invariants:
+       *   pivot >= all in [lo, left).
+       *   pivot <  all in [right, start).
+       */
+      while (left < right) {
+        int mid = (left + right) >>> 1;
+        if (c.compare(pivot, s.getKey(a, mid, key1)) < 0) {
+          right = mid;
+        }
+        else {
+          left = mid + 1;
+        }
+      }
+      assert left == right;
+
+      /*
+       * The invariants still hold: pivot >= all in [lo, left) and
+       * pivot < all in [left, start), so pivot belongs at left.  Note
+       * that if there are elements equal to pivot, left points to the
+       * first slot after them -- that's why this sort is stable.
+       * Slide elements over to make room for pivot.
+       */
+      int n = start - left;  // The number of elements to move
+      // Switch is just an optimization for arraycopy in default case
+      switch (n) {
+        case 2:  {
+          s.copyElement(a, left + 1, a, left + 2);
+          s.copyElement(a, left, a, left + 1);
+          break;
+        }
+        case 1:  {
+          s.copyElement(a, left, a, left + 1);
+          break;
+        }
+        default: s.copyRange(a, left, a, left + 1, n);
+      }
+      s.copyElement(pivotStore, 0, a, left);
+    }
+  }
+
+  /**
+   * Returns the length of the run beginning at the specified position in
+   * the specified array and reverses the run if it is descending (ensuring
+   * that the run will always be ascending when the method returns).
+   *
+   * A run is the longest ascending sequence with:
+   *
+   *    a[lo] <= a[lo + 1] <= a[lo + 2] <= ...
+   *
+   * or the longest descending sequence with:
+   *
+   *    a[lo] >  a[lo + 1] >  a[lo + 2] >  ...
+   *
+   * For its intended use in a stable mergesort, the strictness of the
+   * definition of "descending" is needed so that the call can safely
+   * reverse a descending sequence without violating stability.
+   *
+   * @param a the array in which a run is to be counted and possibly reversed
+   * @param lo index of the first element in the run
+   * @param hi index after the last element that may be contained in the run.
+  It is required that {@code lo < hi}.
+   * @param c the comparator to used for the sort
+   * @return  the length of the run beginning at the specified position in
+   *          the specified array
+   */
+  private int countRunAndMakeAscending(Buffer a, int lo, int hi, Comparator<? super K> c) {
+    assert lo < hi;
+    int runHi = lo + 1;
+    if (runHi == hi) {
+      return 1;
+    }
+
+    K key0 = s.newKey();
+    K key1 = s.newKey();
+
+    // Find end of run, and reverse range if descending
+    if (c.compare(s.getKey(a, runHi++, key0), s.getKey(a, lo, key1)) < 0) { // Descending
+      while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) < 0) {
+        runHi++;
+      }
+      reverseRange(a, lo, runHi);
+    } else {                              // Ascending
+      while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) >= 0) {
+        runHi++;
+      }
+    }
+
+    return runHi - lo;
+  }
+
+  /**
+   * Reverse the specified range of the specified array.
+   *
+   * @param a the array in which a range is to be reversed
+   * @param lo the index of the first element in the range to be reversed
+   * @param hi the index after the last element in the range to be reversed
+   */
+  private void reverseRange(Buffer a, int lo, int hi) {
+    hi--;
+    while (lo < hi) {
+      s.swap(a, lo, hi);
+      lo++;
+      hi--;
+    }
+  }
+
+  /**
+   * Returns the minimum acceptable run length for an array of the specified
+   * length. Natural runs shorter than this will be extended with
+   * {@link #binarySort}.
+   *
+   * Roughly speaking, the computation is:
+   *
+   *  If n < MIN_MERGE, return n (it's too small to bother with fancy stuff).
+   *  Else if n is an exact power of 2, return MIN_MERGE/2.
+   *  Else return an int k, MIN_MERGE/2 <= k <= MIN_MERGE, such that n/k
+   *   is close to, but strictly less than, an exact power of 2.
+   *
+   * For the rationale, see listsort.txt.
+   *
+   * @param n the length of the array to be sorted
+   * @return the length of the minimum run to be merged
+   */
+  private int minRunLength(int n) {
+    assert n >= 0;
+    int r = 0;      // Becomes 1 if any 1 bits are shifted off
+    while (n >= MIN_MERGE) {
+      r |= (n & 1);
+      n >>= 1;
+    }
+    return n + r;
+  }
+
+  private class SortState {
+
+    /**
+     * The Buffer being sorted.
+     */
+    private final Buffer a;
+
+    /**
+     * Length of the sort Buffer.
+     */
+    private final int aLength;
+
+    /**
+     * The comparator for this sort.
+     */
+    private final Comparator<? super K> c;
+
+    /**
+     * When we get into galloping mode, we stay there until both runs win less
+     * often than MIN_GALLOP consecutive times.
+     */
+    private static final int  MIN_GALLOP = 7;
+
+    /**
+     * This controls when we get *into* galloping mode.  It is initialized
+     * to MIN_GALLOP.  The mergeLo and mergeHi methods nudge it higher for
+     * random data, and lower for highly structured data.
+     */
+    private int minGallop = MIN_GALLOP;
+
+    /**
+     * Maximum initial size of tmp array, which is used for merging.  The array
+     * can grow to accommodate demand.
+     *
+     * Unlike Tim's original C version, we do not allocate this much storage
+     * when sorting smaller arrays.  This change was required for performance.
+     */
+    private static final int INITIAL_TMP_STORAGE_LENGTH = 256;
+
+    /**
+     * Temp storage for merges.
+     */
+    private Buffer tmp; // Actual runtime type will be Object[], regardless of T
+
+    /**
+     * Length of the temp storage.
+     */
+    private int tmpLength = 0;
+
+    /**
+     * A stack of pending runs yet to be merged.  Run i starts at
+     * address base[i] and extends for len[i] elements.  It's always
+     * true (so long as the indices are in bounds) that:
+     *
+     *     runBase[i] + runLen[i] == runBase[i + 1]
+     *
+     * so we could cut the storage for this, but it's a minor amount,
+     * and keeping all the info explicit simplifies the code.
+     */
+    private int stackSize = 0;  // Number of pending runs on stack
+    private final int[] runBase;
+    private final int[] runLen;
+
+    /**
+     * Creates a TimSort instance to maintain the state of an ongoing sort.
+     *
+     * @param a the array to be sorted
+     * @param c the comparator to determine the order of the sort
+     */
+    private SortState(Buffer a, Comparator<? super K> c, int len) {
+      this.aLength = len;
+      this.a = a;
+      this.c = c;
+
+      // Allocate temp storage (which may be increased later if necessary)
+      tmpLength = len < 2 * INITIAL_TMP_STORAGE_LENGTH ? len >>> 1 : INITIAL_TMP_STORAGE_LENGTH;
+      tmp = s.allocate(tmpLength);
+
+      /*
+       * Allocate runs-to-be-merged stack (which cannot be expanded).  The
+       * stack length requirements are described in listsort.txt.  The C
+       * version always uses the same stack length (85), but this was
+       * measured to be too expensive when sorting "mid-sized" arrays (e.g.,
+       * 100 elements) in Java.  Therefore, we use smaller (but sufficiently
+       * large) stack lengths for smaller arrays.  The "magic numbers" in the
+       * computation below must be changed if MIN_MERGE is decreased.  See
+       * the MIN_MERGE declaration above for more information.
+       */
+      int stackLen = (len <    120  ?  5 :
+                      len <   1542  ? 10 :
+                      len < 119151  ? 19 : 40);
+      runBase = new int[stackLen];
+      runLen = new int[stackLen];
+    }
+
+    /**
+     * Pushes the specified run onto the pending-run stack.
+     *
+     * @param runBase index of the first element in the run
+     * @param runLen  the number of elements in the run
+     */
+    private void pushRun(int runBase, int runLen) {
+      this.runBase[stackSize] = runBase;
+      this.runLen[stackSize] = runLen;
+      stackSize++;
+    }
+
+    /**
+     * Examines the stack of runs waiting to be merged and merges adjacent runs
+     * until the stack invariants are reestablished:
+     *
+     *     1. runLen[i - 3] > runLen[i - 2] + runLen[i - 1]
+     *     2. runLen[i - 2] > runLen[i - 1]
+     *
+     * This method is called each time a new run is pushed onto the stack,
+     * so the invariants are guaranteed to hold for i < stackSize upon
+     * entry to the method.
+     */
+    private void mergeCollapse() {
+      while (stackSize > 1) {
+        int n = stackSize - 2;
+        if ((n >= 1 && runLen[n - 1] <= runLen[n] + runLen[n + 1])
+            || (n >= 2 && runLen[n - 2] <= runLen[n] + runLen[n - 1])) {
+          if (runLen[n - 1] < runLen[n + 1]) {
+            n--;
+          }
+        } else if (runLen[n] > runLen[n + 1]) {
+          break; // Invariant is established
+        }
+        mergeAt(n);
+      }
+    }
+
+    /**
+     * Merges all runs on the stack until only one remains.  This method is
+     * called once, to complete the sort.
+     */
+    private void mergeForceCollapse() {
+      while (stackSize > 1) {
+        int n = stackSize - 2;
+        if (n > 0 && runLen[n - 1] < runLen[n + 1]) {
+          n--;
+        }
+        mergeAt(n);
+      }
+    }
+
+    /**
+     * Merges the two runs at stack indices i and i+1.  Run i must be
+     * the penultimate or antepenultimate run on the stack.  In other words,
+     * i must be equal to stackSize-2 or stackSize-3.
+     *
+     * @param i stack index of the first of the two runs to merge
+     */
+    private void mergeAt(int i) {
+      assert stackSize >= 2;
+      assert i >= 0;
+      assert i == stackSize - 2 || i == stackSize - 3;
+
+      int base1 = runBase[i];
+      int len1 = runLen[i];
+      int base2 = runBase[i + 1];
+      int len2 = runLen[i + 1];
+      assert len1 > 0 && len2 > 0;
+      assert base1 + len1 == base2;
+
+      /*
+       * Record the length of the combined runs; if i is the 3rd-last
+       * run now, also slide over the last run (which isn't involved
+       * in this merge).  The current run (i+1) goes away in any case.
+       */
+      runLen[i] = len1 + len2;
+      if (i == stackSize - 3) {
+        runBase[i + 1] = runBase[i + 2];
+        runLen[i + 1] = runLen[i + 2];
+      }
+      stackSize--;
+
+      K key0 = s.newKey();
+
+      /*
+       * Find where the first element of run2 goes in run1. Prior elements
+       * in run1 can be ignored (because they're already in place).
+       */
+      int k = gallopRight(s.getKey(a, base2, key0), a, base1, len1, 0, c);
+      assert k >= 0;
+      base1 += k;
+      len1 -= k;
+      if (len1 == 0) {
+        return;
+      }
+
+      /*
+       * Find where the last element of run1 goes in run2. Subsequent elements
+       * in run2 can be ignored (because they're already in place).
+       */
+      len2 = gallopLeft(s.getKey(a, base1 + len1 - 1, key0), a, base2, len2, len2 - 1, c);
+      assert len2 >= 0;
+      if (len2 == 0) {
+        return;
+      }
+
+      // Merge remaining runs, using tmp array with min(len1, len2) elements
+      if (len1 <= len2) {
+        mergeLo(base1, len1, base2, len2);
+      }
+      else {
+        mergeHi(base1, len1, base2, len2);
+      }
+    }
+
+    /**
+     * Locates the position at which to insert the specified key into the
+     * specified sorted range; if the range contains an element equal to key,
+     * returns the index of the leftmost equal element.
+     *
+     * @param key the key whose insertion point to search for
+     * @param a the array in which to search
+     * @param base the index of the first element in the range
+     * @param len the length of the range; must be > 0
+     * @param hint the index at which to begin the search, 0 <= hint < n.
+     *     The closer hint is to the result, the faster this method will run.
+     * @param c the comparator used to order the range, and to search
+     * @return the int k,  0 <= k <= n such that a[b + k - 1] < key <= a[b + k],
+     *    pretending that a[b - 1] is minus infinity and a[b + n] is infinity.
+     *    In other words, key belongs at index b + k; or in other words,
+     *    the first k elements of a should precede key, and the last n - k
+     *    should follow it.
+     */
+    private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator<? super K> c) {
+      assert len > 0 && hint >= 0 && hint < len;
+      int lastOfs = 0;
+      int ofs = 1;
+      K key0 = s.newKey();
+
+      if (c.compare(key, s.getKey(a, base + hint, key0)) > 0) {
+        // Gallop right until a[base+hint+lastOfs] < key <= a[base+hint+ofs]
+        int maxOfs = len - hint;
+        while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key0)) > 0) {
+          lastOfs = ofs;
+          ofs = (ofs << 1) + 1;
+          if (ofs <= 0) {  // int overflow
+            ofs = maxOfs;
+          }
+        }
+        if (ofs > maxOfs) {
+          ofs = maxOfs;
+        }
+
+        // Make offsets relative to base
+        lastOfs += hint;
+        ofs += hint;
+      } else { // key <= a[base + hint]
+        // Gallop left until a[base+hint-ofs] < key <= a[base+hint-lastOfs]
+        final int maxOfs = hint + 1;
+        while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key0)) <= 0) {
+          lastOfs = ofs;
+          ofs = (ofs << 1) + 1;
+          if (ofs <= 0) {  // int overflow
+            ofs = maxOfs;
+          }
+        }
+        if (ofs > maxOfs) {
+          ofs = maxOfs;
+        }
+
+        // Make offsets relative to base
+        int tmp = lastOfs;
+        lastOfs = hint - ofs;
+        ofs = hint - tmp;
+      }
+      assert -1 <= lastOfs && lastOfs < ofs && ofs <= len;
+
+      /*
+       * Now a[base+lastOfs] < key <= a[base+ofs], so key belongs somewhere
+       * to the right of lastOfs but no farther right than ofs.  Do a binary
+       * search, with invariant a[base + lastOfs - 1] < key <= a[base + ofs].
+       */
+      lastOfs++;
+      while (lastOfs < ofs) {
+        int m = lastOfs + ((ofs - lastOfs) >>> 1);
+
+        if (c.compare(key, s.getKey(a, base + m, key0)) > 0) {
+          lastOfs = m + 1;  // a[base + m] < key
+        }
+        else {
+          ofs = m;          // key <= a[base + m]
+        }
+      }
+      assert lastOfs == ofs;    // so a[base + ofs - 1] < key <= a[base + ofs]
+      return ofs;
+    }
+
+    /**
+     * Like gallopLeft, except that if the range contains an element equal to
+     * key, gallopRight returns the index after the rightmost equal element.
+     *
+     * @param key the key whose insertion point to search for
+     * @param a the array in which to search
+     * @param base the index of the first element in the range
+     * @param len the length of the range; must be > 0
+     * @param hint the index at which to begin the search, 0 <= hint < n.
+     *     The closer hint is to the result, the faster this method will run.
+     * @param c the comparator used to order the range, and to search
+     * @return the int k,  0 <= k <= n such that a[b + k - 1] <= key < a[b + k]
+     */
+    private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator<? super K> c) {
+      assert len > 0 && hint >= 0 && hint < len;
+
+      int ofs = 1;
+      int lastOfs = 0;
+      K key1 = s.newKey();
+
+      if (c.compare(key, s.getKey(a, base + hint, key1)) < 0) {
+        // Gallop left until a[b+hint - ofs] <= key < a[b+hint - lastOfs]
+        int maxOfs = hint + 1;
+        while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key1)) < 0) {
+          lastOfs = ofs;
+          ofs = (ofs << 1) + 1;
+          if (ofs <= 0) {  // int overflow
+            ofs = maxOfs;
+          }
+        }
+        if (ofs > maxOfs) {
+          ofs = maxOfs;
+        }
+
+        // Make offsets relative to b
+        int tmp = lastOfs;
+        lastOfs = hint - ofs;
+        ofs = hint - tmp;
+      } else { // a[b + hint] <= key
+        // Gallop right until a[b+hint + lastOfs] <= key < a[b+hint + ofs]
+        int maxOfs = len - hint;
+        while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key1)) >= 0) {
+          lastOfs = ofs;
+          ofs = (ofs << 1) + 1;
+          if (ofs <= 0) {  // int overflow
+            ofs = maxOfs;
+          }
+        }
+        if (ofs > maxOfs) {
+          ofs = maxOfs;
+        }
+
+        // Make offsets relative to b
+        lastOfs += hint;
+        ofs += hint;
+      }
+      assert -1 <= lastOfs && lastOfs < ofs && ofs <= len;
+
+      /*
+       * Now a[b + lastOfs] <= key < a[b + ofs], so key belongs somewhere to
+       * the right of lastOfs but no farther right than ofs.  Do a binary
+       * search, with invariant a[b + lastOfs - 1] <= key < a[b + ofs].
+       */
+      lastOfs++;
+      while (lastOfs < ofs) {
+        int m = lastOfs + ((ofs - lastOfs) >>> 1);
+
+        if (c.compare(key, s.getKey(a, base + m, key1)) < 0) {
+          ofs = m;          // key < a[b + m]
+        }
+        else {
+          lastOfs = m + 1;  // a[b + m] <= key
+        }
+      }
+      assert lastOfs == ofs;    // so a[b + ofs - 1] <= key < a[b + ofs]
+      return ofs;
+    }
+
+    /**
+     * Merges two adjacent runs in place, in a stable fashion.  The first
+     * element of the first run must be greater than the first element of the
+     * second run (a[base1] > a[base2]), and the last element of the first run
+     * (a[base1 + len1-1]) must be greater than all elements of the second run.
+     *
+     * For performance, this method should be called only when len1 <= len2;
+     * its twin, mergeHi should be called if len1 >= len2.  (Either method
+     * may be called if len1 == len2.)
+     *
+     * @param base1 index of first element in first run to be merged
+     * @param len1  length of first run to be merged (must be > 0)
+     * @param base2 index of first element in second run to be merged
+     *        (must be aBase + aLen)
+     * @param len2  length of second run to be merged (must be > 0)
+     */
+    private void mergeLo(int base1, int len1, int base2, int len2) {
+      assert len1 > 0 && len2 > 0 && base1 + len1 == base2;
+
+      // Copy first run into temp array
+      Buffer a = this.a; // For performance
+      Buffer tmp = ensureCapacity(len1);
+      s.copyRange(a, base1, tmp, 0, len1);
+
+      int cursor1 = 0;       // Indexes into tmp array
+      int cursor2 = base2;   // Indexes int a
+      int dest = base1;      // Indexes int a
+
+      // Move first element of second run and deal with degenerate cases
+      s.copyElement(a, cursor2++, a, dest++);
+      if (--len2 == 0) {
+        s.copyRange(tmp, cursor1, a, dest, len1);
+        return;
+      }
+      if (len1 == 1) {
+        s.copyRange(a, cursor2, a, dest, len2);
+        s.copyElement(tmp, cursor1, a, dest + len2); // Last elt of run 1 to end of merge
+        return;
+      }
+
+      K key0 = s.newKey();
+      K key1 = s.newKey();
+
+      Comparator<? super K> c = this.c;  // Use local variable for performance
+      int minGallop = this.minGallop;    //  "    "       "     "      "
+      outer:
+      while (true) {
+        int count1 = 0; // Number of times in a row that first run won
+        int count2 = 0; // Number of times in a row that second run won
+
+        /*
+         * Do the straightforward thing until (if ever) one run starts
+         * winning consistently.
+         */
+        do {
+          assert len1 > 1 && len2 > 0;
+          if (c.compare(s.getKey(a, cursor2, key0), s.getKey(tmp, cursor1, key1)) < 0) {
+            s.copyElement(a, cursor2++, a, dest++);
+            count2++;
+            count1 = 0;
+            if (--len2 == 0) {
+              break outer;
+            }
+          } else {
+            s.copyElement(tmp, cursor1++, a, dest++);
+            count1++;
+            count2 = 0;
+            if (--len1 == 1) {
+              break outer;
+            }
+          }
+        } while ((count1 | count2) < minGallop);
+
+        /*
+         * One run is winning so consistently that galloping may be a
+         * huge win. So try that, and continue galloping until (if ever)
+         * neither run appears to be winning consistently anymore.
+         */
+        do {
+          assert len1 > 1 && len2 > 0;
+          count1 = gallopRight(s.getKey(a, cursor2, key0), tmp, cursor1, len1, 0, c);
+          if (count1 != 0) {
+            s.copyRange(tmp, cursor1, a, dest, count1);
+            dest += count1;
+            cursor1 += count1;
+            len1 -= count1;
+            if (len1 <= 1) { // len1 == 1 || len1 == 0
+              break outer;
+            }
+          }
+          s.copyElement(a, cursor2++, a, dest++);
+          if (--len2 == 0) {
+            break outer;
+          }
+
+          count2 = gallopLeft(s.getKey(tmp, cursor1, key0), a, cursor2, len2, 0, c);
+          if (count2 != 0) {
+            s.copyRange(a, cursor2, a, dest, count2);
+            dest += count2;
+            cursor2 += count2;
+            len2 -= count2;
+            if (len2 == 0) {
+              break outer;
+            }
+          }
+          s.copyElement(tmp, cursor1++, a, dest++);
+          if (--len1 == 1) {
+            break outer;
+          }
+          minGallop--;
+        } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP);
+        if (minGallop < 0) {
+          minGallop = 0;
+        }
+        minGallop += 2;  // Penalize for leaving gallop mode
+      }  // End of "outer" loop
+      this.minGallop = minGallop < 1 ? 1 : minGallop;  // Write back to field
+
+      if (len1 == 1) {
+        assert len2 > 0;
+        s.copyRange(a, cursor2, a, dest, len2);
+        s.copyElement(tmp, cursor1, a, dest + len2); //  Last elt of run 1 to end of merge
+      } else if (len1 == 0) {
+        throw new IllegalArgumentException(
+            "Comparison method violates its general contract!");
+      } else {
+        assert len2 == 0;
+        assert len1 > 1;
+        s.copyRange(tmp, cursor1, a, dest, len1);
+      }
+    }
+
+    /**
+     * Like mergeLo, except that this method should be called only if
+     * len1 >= len2; mergeLo should be called if len1 <= len2.  (Either method
+     * may be called if len1 == len2.)
+     *
+     * @param base1 index of first element in first run to be merged
+     * @param len1  length of first run to be merged (must be > 0)
+     * @param base2 index of first element in second run to be merged
+     *        (must be aBase + aLen)
+     * @param len2  length of second run to be merged (must be > 0)
+     */
+    private void mergeHi(int base1, int len1, int base2, int len2) {
+      assert len1 > 0 && len2 > 0 && base1 + len1 == base2;
+
+      // Copy second run into temp array
+      Buffer a = this.a; // For performance
+      Buffer tmp = ensureCapacity(len2);
+      s.copyRange(a, base2, tmp, 0, len2);
+
+      int cursor1 = base1 + len1 - 1;  // Indexes into a
+      int cursor2 = len2 - 1;          // Indexes into tmp array
+      int dest = base2 + len2 - 1;     // Indexes into a
+
+      K key0 = s.newKey();
+      K key1 = s.newKey();
+
+      // Move last element of first run and deal with degenerate cases
+      s.copyElement(a, cursor1--, a, dest--);
+      if (--len1 == 0) {
+        s.copyRange(tmp, 0, a, dest - (len2 - 1), len2);
+        return;
+      }
+      if (len2 == 1) {
+        dest -= len1;
+        cursor1 -= len1;
+        s.copyRange(a, cursor1 + 1, a, dest + 1, len1);
+        s.copyElement(tmp, cursor2, a, dest);
+        return;
+      }
+
+      Comparator<? super K> c = this.c;  // Use local variable for performance
+      int minGallop = this.minGallop;    //  "    "       "     "      "
+      outer:
+      while (true) {
+        int count1 = 0; // Number of times in a row that first run won
+        int count2 = 0; // Number of times in a row that second run won
+
+        /*
+         * Do the straightforward thing until (if ever) one run
+         * appears to win consistently.
+         */
+        do {
+          assert len1 > 0 && len2 > 1;
+          if (c.compare(s.getKey(tmp, cursor2, key0), s.getKey(a, cursor1, key1)) < 0) {
+            s.copyElement(a, cursor1--, a, dest--);
+            count1++;
+            count2 = 0;
+            if (--len1 == 0) {
+              break outer;
+            }
+          } else {
+            s.copyElement(tmp, cursor2--, a, dest--);
+            count2++;
+            count1 = 0;
+            if (--len2 == 1) {
+              break outer;
+            }
+          }
+        } while ((count1 | count2) < minGallop);
+
+        /*
+         * One run is winning so consistently that galloping may be a
+         * huge win. So try that, and continue galloping until (if ever)
+         * neither run appears to be winning consistently anymore.
+         */
+        do {
+          assert len1 > 0 && len2 > 1;
+          count1 = len1 - gallopRight(s.getKey(tmp, cursor2, key0), a, base1, len1, len1 - 1, c);
+          if (count1 != 0) {
+            dest -= count1;
+            cursor1 -= count1;
+            len1 -= count1;
+            s.copyRange(a, cursor1 + 1, a, dest + 1, count1);
+            if (len1 == 0) {
+              break outer;
+            }
+          }
+          s.copyElement(tmp, cursor2--, a, dest--);
+          if (--len2 == 1) {
+            break outer;
+          }
+
+          count2 = len2 - gallopLeft(s.getKey(a, cursor1, key0), tmp, 0, len2, len2 - 1, c);
+          if (count2 != 0) {
+            dest -= count2;
+            cursor2 -= count2;
+            len2 -= count2;
+            s.copyRange(tmp, cursor2 + 1, a, dest + 1, count2);
+            if (len2 <= 1) { // len2 == 1 || len2 == 0
+              break outer;
+            }
+          }
+          s.copyElement(a, cursor1--, a, dest--);
+          if (--len1 == 0) {
+            break outer;
+          }
+          minGallop--;
+        } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP);
+        if (minGallop < 0) {
+          minGallop = 0;
+        }
+        minGallop += 2;  // Penalize for leaving gallop mode
+      }  // End of "outer" loop
+      this.minGallop = minGallop < 1 ? 1 : minGallop;  // Write back to field
+
+      if (len2 == 1) {
+        assert len1 > 0;
+        dest -= len1;
+        cursor1 -= len1;
+        s.copyRange(a, cursor1 + 1, a, dest + 1, len1);
+        s.copyElement(tmp, cursor2, a, dest); // Move first elt of run2 to front of merge
+      } else if (len2 == 0) {
+        throw new IllegalArgumentException(
+            "Comparison method violates its general contract!");
+      } else {
+        assert len1 == 0;
+        assert len2 > 0;
+        s.copyRange(tmp, 0, a, dest - (len2 - 1), len2);
+      }
+    }
+
+    /**
+     * Ensures that the external array tmp has at least the specified
+     * number of elements, increasing its size if necessary.  The size
+     * increases exponentially to ensure amortized linear time complexity.
+     *
+     * @param minCapacity the minimum required capacity of the tmp array
+     * @return tmp, whether or not it grew
+     */
+    private Buffer ensureCapacity(int minCapacity) {
+      if (tmpLength < minCapacity) {
+        // Compute smallest power of 2 > minCapacity
+        int newSize = minCapacity;
+        newSize |= newSize >> 1;
+        newSize |= newSize >> 2;
+        newSize |= newSize >> 4;
+        newSize |= newSize >> 8;
+        newSize |= newSize >> 16;
+        newSize++;
+
+        if (newSize < 0) { // Not bloody likely!
+          newSize = minCapacity;
+        }
+        else {
+          newSize = Math.min(newSize, aLength >>> 1);
+        }
+
+        tmp = s.allocate(newSize);
+        tmpLength = newSize;
+      }
+      return tmp;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/sort/UnsafeIntSortDataFormat.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/sort/UnsafeIntSortDataFormat.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/sort/UnsafeIntSortDataFormat.java
new file mode 100644
index 0000000..92962d9
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/unsafe/sort/UnsafeIntSortDataFormat.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.processing.loading.sort.unsafe.sort;
+
+import org.apache.carbondata.core.memory.IntPointerBuffer;
+import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
+import org.apache.carbondata.processing.loading.sort.unsafe.holder.UnsafeCarbonRow;
+
+import org.apache.spark.util.collection.SortDataFormat;
+
+/**
+ * Interface implementation for utilities to sort the data.
+ */
+public class UnsafeIntSortDataFormat
+    extends SortDataFormat<UnsafeCarbonRow, IntPointerBuffer> {
+
+  private UnsafeCarbonRowPage page;
+
+  public UnsafeIntSortDataFormat(UnsafeCarbonRowPage page) {
+    this.page = page;
+  }
+
+  @Override public UnsafeCarbonRow getKey(IntPointerBuffer data, int pos) {
+    // Since we re-use keys, this method shouldn't be called.
+    throw new UnsupportedOperationException();
+  }
+
+  @Override public UnsafeCarbonRow newKey() {
+    return new UnsafeCarbonRow();
+  }
+
+  @Override
+  public UnsafeCarbonRow getKey(IntPointerBuffer data, int pos, UnsafeCarbonRow reuse) {
+    reuse.address = data.get(pos) + page.getDataBlock().getBaseOffset();
+    return reuse;
+  }
+
+  @Override public void swap(IntPointerBuffer data, int pos0, int pos1) {
+    int tempPointer = data.get(pos0);
+    data.set(pos0, data.get(pos1));
+    data.set(pos1, tempPointer);
+  }
+
+  @Override
+  public void copyElement(IntPointerBuffer src, int srcPos, IntPointerBuffer dst, int dstPos) {
+    dst.set(dstPos, src.get(srcPos));
+  }
+
+  @Override
+  public void copyRange(IntPointerBuffer src, int srcPos, IntPointerBuffer dst, int dstPos,
+      int length) {
+    System.arraycopy(src.getPointerBlock(), srcPos, dst.getPointerBlock(), dstPos, length);
+  }
+
+  @Override public IntPointerBuffer allocate(int length) {
+    return new IntPointerBuffer(length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..7007160
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/CarbonRowDataWriterProcessorStepImpl.java
@@ -0,0 +1,299 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Iterator;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+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.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.datastore.row.WriteStepRowUtil;
+import org.apache.carbondata.core.keygenerator.KeyGenException;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.exception.BadRecordFoundException;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
+import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
+import org.apache.carbondata.processing.store.CarbonFactHandler;
+import org.apache.carbondata.processing.store.CarbonFactHandlerFactory;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+/**
+ * It reads data from sorted files which are generated in previous sort step.
+ * And it writes data to carbondata file. It also generates mdk key while writing to carbondata file
+ */
+public class CarbonRowDataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CarbonRowDataWriterProcessorStepImpl.class.getName());
+
+  private int dimensionWithComplexCount;
+
+  private int noDictWithComplextCount;
+
+  private boolean[] isNoDictionaryDimensionColumn;
+
+  private DataType[] measureDataType;
+
+  private int dimensionCount;
+
+  private int measureCount;
+
+  private long[] readCounter;
+
+  private long[] writeCounter;
+
+  private int outputLength;
+
+  private CarbonTableIdentifier tableIdentifier;
+
+  private String tableName;
+
+  public CarbonRowDataWriterProcessorStepImpl(CarbonDataLoadConfiguration configuration,
+      AbstractDataLoadProcessorStep child) {
+    super(configuration, child);
+  }
+
+  @Override public DataField[] getOutput() {
+    return child.getOutput();
+  }
+
+  @Override public void initialize() throws IOException {
+    super.initialize();
+    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;
+  }
+
+  @Override public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
+    final Iterator<CarbonRowBatch>[] iterators = child.execute();
+    tableIdentifier = configuration.getTableIdentifier().getCarbonTableIdentifier();
+    tableName = tableIdentifier.getTableName();
+    try {
+      readCounter = new long[iterators.length];
+      writeCounter = new long[iterators.length];
+      dimensionWithComplexCount = configuration.getDimensionCount();
+      noDictWithComplextCount =
+          configuration.getNoDictionaryCount() + configuration.getComplexColumnCount();
+      dimensionCount = configuration.getDimensionCount() - noDictWithComplextCount;
+      isNoDictionaryDimensionColumn =
+          CarbonDataProcessorUtil.getNoDictionaryMapping(configuration.getDataFields());
+      measureDataType = configuration.getMeasureDataType();
+      CarbonFactDataHandlerModel dataHandlerModel = CarbonFactDataHandlerModel
+          .createCarbonFactDataHandlerModel(configuration,
+              getStoreLocation(tableIdentifier, String.valueOf(0)), 0, 0);
+      measureCount = dataHandlerModel.getMeasureCount();
+      outputLength = measureCount + (this.noDictWithComplextCount > 0 ? 1 : 0) + 1;
+      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+              System.currentTimeMillis());
+
+      if (iterators.length == 1) {
+        doExecute(iterators[0], 0, 0);
+      } else {
+        ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
+        Future[] futures = new Future[iterators.length];
+        for (int i = 0; i < iterators.length; i++) {
+          futures[i] = executorService.submit(new DataWriterRunnable(iterators[i], i));
+        }
+        for (Future future : futures) {
+          future.get();
+        }
+      }
+    } catch (CarbonDataWriterException e) {
+      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
+      throw new CarbonDataLoadingException(
+          "Error while initializing data handler : " + e.getMessage());
+    } catch (Exception e) {
+      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
+      if (e instanceof BadRecordFoundException) {
+        throw new BadRecordFoundException(e.getMessage(), e);
+      }
+      throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage(), e);
+    }
+    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);
+    CarbonFactHandler dataHandler = null;
+    boolean rowsNotExist = true;
+    while (iterator.hasNext()) {
+      if (rowsNotExist) {
+        rowsNotExist = false;
+        dataHandler = CarbonFactHandlerFactory
+            .createCarbonFactHandler(model, CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
+        dataHandler.initialise();
+      }
+      processBatch(iterator.next(), dataHandler, iteratorIndex);
+    }
+    if (!rowsNotExist) {
+      finish(dataHandler, iteratorIndex);
+    }
+  }
+
+  @Override protected String getStepName() {
+    return "Data Writer";
+  }
+
+  private void finish(CarbonFactHandler dataHandler, int iteratorIndex) {
+    try {
+      dataHandler.finish();
+    } catch (Exception e) {
+      LOGGER.error(e, "Failed for table: " + tableName + " in  finishing data handler");
+    }
+    LOGGER.info("Record Processed For table: " + tableName);
+    String logMessage =
+        "Finished Carbon DataWriterProcessorStepImpl: Read: " + readCounter[iteratorIndex]
+            + ": Write: " + readCounter[iteratorIndex];
+    LOGGER.info(logMessage);
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
+    processingComplete(dataHandler);
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+            System.currentTimeMillis());
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
+  }
+
+  private void processingComplete(CarbonFactHandler dataHandler) throws CarbonDataLoadingException {
+    if (null != dataHandler) {
+      try {
+        dataHandler.closeHandler();
+      } catch (CarbonDataWriterException e) {
+        LOGGER.error(e, e.getMessage());
+        throw new CarbonDataLoadingException(e.getMessage());
+      } catch (Exception e) {
+        LOGGER.error(e, e.getMessage());
+        throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage());
+      }
+    }
+  }
+
+  /**
+   * convert input CarbonRow to output CarbonRow
+   * e.g. There is a table as following,
+   * the number of dictionary dimensions is a,
+   * the number of no-dictionary dimensions is b,
+   * the number of complex dimensions is c,
+   * the number of measures is d.
+   * input CarbonRow format:  the length of Object[] data is a+b+c+d, the number of all columns.
+   * ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
+   * | Part                     | Object item                    | describe                 |
+   * ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
+   * | Object[0 ~ a+b-1]        | Integer, byte[], Integer, ...  | dict + no dict dimensions|
+   * ----------------------------------------------------------------------------------------
+   * | Object[a+b ~ a+b+c-1]    | byte[], byte[], ...            | complex dimensions       |
+   * ----------------------------------------------------------------------------------------
+   * | Object[a+b+c ~ a+b+c+d-1]| int, byte[], ...               | measures                 |
+   * ----------------------------------------------------------------------------------------
+   * output CarbonRow format: the length of object[] data is d + (b+c>0?1:0) + 1.
+   * ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
+   * | Part                     | Object item                    | describe                 |
+   * ++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++++
+   * | Object[d+1]              | byte[]                         | mdkey                    |
+   * ----------------------------------------------------------------------------------------
+   * | Object[d]                | byte[b+c][]                    | no dict + complex dim    |
+   * ----------------------------------------------------------------------------------------
+   * | Object[0 ~ d-1]          | int, byte[], ...               | measures                 |
+   * ----------------------------------------------------------------------------------------
+   *
+   * @param row
+   * @return
+   */
+  private CarbonRow convertRow(CarbonRow row) throws KeyGenException {
+    int dictIndex = 0;
+    int nonDicIndex = 0;
+    int[] dim = new int[this.dimensionCount];
+    byte[][] nonDicArray = new byte[this.noDictWithComplextCount][];
+    // read dimension values
+    int dimCount = 0;
+    for (; dimCount < isNoDictionaryDimensionColumn.length; dimCount++) {
+      if (isNoDictionaryDimensionColumn[dimCount]) {
+        nonDicArray[nonDicIndex++] = (byte[]) row.getObject(dimCount);
+      } else {
+        dim[dictIndex++] = (int) row.getObject(dimCount);
+      }
+    }
+
+    for (; dimCount < this.dimensionWithComplexCount; dimCount++) {
+      nonDicArray[nonDicIndex++] = (byte[]) row.getObject(dimCount);
+    }
+
+    Object[] measures = new Object[measureCount];
+    for (int i = 0; i < this.measureCount; i++) {
+      measures[i] = row.getObject(i + this.dimensionWithComplexCount);
+    }
+
+    return WriteStepRowUtil.fromColumnCategory(dim, nonDicArray, measures);
+  }
+
+  private void processBatch(CarbonRowBatch batch, CarbonFactHandler dataHandler, int iteratorIndex)
+      throws CarbonDataLoadingException {
+    try {
+      while (batch.hasNext()) {
+        CarbonRow row = batch.next();
+        CarbonRow converted = convertRow(row);
+        dataHandler.addDataToStore(converted);
+        readCounter[iteratorIndex]++;
+      }
+      writeCounter[iteratorIndex] += batch.getSize();
+    } catch (Exception e) {
+      throw new CarbonDataLoadingException("unable to generate the mdkey", e);
+    }
+    rowCounter.getAndAdd(batch.getSize());
+  }
+
+  @Override protected CarbonRow processRow(CarbonRow row) {
+    return null;
+  }
+
+  class DataWriterRunnable implements Runnable {
+
+    private Iterator<CarbonRowBatch> iterator;
+    private int iteratorIndex = 0;
+
+    DataWriterRunnable(Iterator<CarbonRowBatch> iterator, int iteratorIndex) {
+      this.iterator = iterator;
+      this.iteratorIndex = iteratorIndex;
+    }
+
+    @Override public void run() {
+      doExecute(this.iterator, 0, iteratorIndex);
+    }
+  }
+}


[49/50] [abbrv] carbondata git commit: [CARBONDATA-1173] Stream ingestion - write path framework

Posted by ja...@apache.org.
[CARBONDATA-1173] Stream ingestion - write path framework

This closes #1064


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

Branch: refs/heads/streaming_ingest
Commit: 441907ee84801de264916af7633c649a8e4a13f4
Parents: ad25ffc
Author: Aniket Adnaik <an...@gmail.com>
Authored: Thu Jun 15 11:57:43 2017 -0700
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Oct 10 11:03:26 2017 +0800

----------------------------------------------------------------------
 .../streaming/CarbonStreamingCommitInfo.java    | 108 ++++++++++
 .../streaming/CarbonStreamingConstants.java     |  25 +++
 .../streaming/CarbonStreamingMetaStore.java     |  40 ++++
 .../streaming/CarbonStreamingMetaStoreImpl.java |  56 ++++++
 .../core/util/path/CarbonTablePath.java         |  10 +
 .../streaming/CarbonStreamingOutputFormat.java  |  66 +++++++
 .../streaming/CarbonStreamingRecordWriter.java  | 196 +++++++++++++++++++
 .../org/apache/spark/sql/CarbonSource.scala     |  39 +++-
 .../CarbonStreamingOutpurWriteFactory.scala     |  88 +++++++++
 .../streaming/CarbonStreamingOutputWriter.scala |  98 ++++++++++
 10 files changed, 719 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java b/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java
new file mode 100644
index 0000000..6cf303a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingCommitInfo.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.streaming;
+
+/**
+ * Commit info for streaming writes
+ * The commit info can be used to recover valid offset in the file
+ * in the case of write failure.
+ */
+public class CarbonStreamingCommitInfo {
+
+  private String dataBase;
+
+  private String table;
+
+  private long commitTime;
+
+  private long segmentID;
+
+  private String partitionID;
+
+  private long batchID;
+
+  private String fileOffset;
+
+  private long transactionID;     // future use
+
+  public  CarbonStreamingCommitInfo(
+
+      String dataBase,
+
+      String table,
+
+      long commitTime,
+
+      long segmentID,
+
+      String partitionID,
+
+      long batchID) {
+
+    this.dataBase = dataBase;
+
+    this.table = table;
+
+    this.commitTime = commitTime;
+
+    this.segmentID = segmentID;
+
+    this.partitionID = partitionID;
+
+    this.batchID = batchID;
+
+    this.transactionID = -1;
+  }
+
+  public String getDataBase() {
+    return dataBase;
+  }
+
+  public String getTable() {
+    return table;
+  }
+
+  public long getCommitTime() {
+    return commitTime;
+  }
+
+  public long getSegmentID() {
+    return segmentID;
+  }
+
+  public String getPartitionID() {
+    return partitionID;
+  }
+
+  public long getBatchID() {
+    return batchID;
+  }
+
+  public String getFileOffset() {
+    return fileOffset;
+  }
+
+  public long getTransactionID() {
+    return transactionID;
+  }
+
+  @Override
+  public String toString() {
+    return dataBase + "." + table + "." + segmentID + "$" + partitionID;
+  }
+}

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

http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingMetaStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingMetaStore.java b/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingMetaStore.java
new file mode 100644
index 0000000..fa3746c
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingMetaStore.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.core.streaming;
+
+
+import java.io.IOException;
+
+/**
+ * Generic interface for storing commit info for streaming ingest
+ */
+public interface CarbonStreamingMetaStore {
+
+  public CarbonStreamingCommitInfo getStreamingCommitInfo(
+          String dataBase,
+          String table,
+          long segmentID,
+          String partitionID) throws IOException;
+
+  public void updateStreamingCommitInfo(
+          CarbonStreamingMetaStore commitInfo) throws IOException;
+
+  public void recoverStreamingData(
+          CarbonStreamingCommitInfo commitInfo) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingMetaStoreImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingMetaStoreImpl.java b/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingMetaStoreImpl.java
new file mode 100644
index 0000000..0afe962
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/streaming/CarbonStreamingMetaStoreImpl.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.streaming;
+
+import java.io.IOException;
+
+/**
+ *  JSON format can be used to store the metadata
+ */
+public class CarbonStreamingMetaStoreImpl implements CarbonStreamingMetaStore {
+
+  /**
+   * get commit info from metastore
+   */
+  public CarbonStreamingCommitInfo getStreamingCommitInfo(
+          String dataBase,
+          String table,
+          long segmentID,
+          String partitionID) throws IOException {
+
+    return null;
+
+  }
+
+  /**
+   * Update commit info in metastore
+   */
+  public void updateStreamingCommitInfo(
+          CarbonStreamingMetaStore commitInfo) throws IOException {
+
+  }
+
+  /**
+   * Recover streaming data using valid offset in commit info
+   */
+  public void recoverStreamingData(
+          CarbonStreamingCommitInfo commitInfo) throws IOException {
+
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/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 0910afc..8f4fa26 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
@@ -51,6 +51,16 @@ public class CarbonTablePath extends Path {
   protected static final String INDEX_FILE_EXT = ".carbonindex";
   protected static final String DELETE_DELTA_FILE_EXT = ".deletedelta";
 
+  /**
+   * Streaming ingest related paths
+   */
+  protected static final String STREAM_PREFIX = "Streaming";
+  protected static final String STREAM_FILE_NAME_EXT = ".carbondata.stream";
+  protected static final String STREAM_FILE_BEING_WRITTEN = "in-progress.carbondata.stream";
+  protected static final String STREAM_FILE_BEING_WRITTEN_META = "in-progress.meta";
+  protected static final String STREAM_COMPACTION_STATUS = "streaming_compaction_status";
+  protected static final String STREAM_FILE_LOCK = "streaming_in_use.lock";
+
   protected String tablePath;
   protected CarbonTableIdentifier carbonTableIdentifier;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamingOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamingOutputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamingOutputFormat.java
new file mode 100644
index 0000000..350684e
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamingOutputFormat.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.hadoop.streaming;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.streaming.CarbonStreamingConstants;
+import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+
+/**
+ * Output format to write streaming data to carbondata file
+ *
+ * @param <V> - type of record
+ */
+public class CarbonStreamingOutputFormat<K, V> extends FileOutputFormat<K, V> {
+
+  public static long getBlockSize(Configuration conf) {
+    return conf.getLong("dfs.block.size",
+            CarbonStreamingConstants.DEFAULT_CARBON_STREAM_FILE_BLOCK_SIZE);
+  }
+
+  public static void setBlockSize(Configuration conf, long blockSize) {
+    conf.setLong("dfs.block.size", blockSize);
+  }
+
+  /**
+   * When getRecordWriter may need to override
+   * to provide correct path including streaming segment name
+   */
+  @Override
+  public CarbonStreamingRecordWriter<K, V> getRecordWriter(TaskAttemptContext job)
+          throws IOException, InterruptedException {
+
+    Configuration conf = job.getConfiguration();
+
+    String keyValueSeparator = conf.get(
+            CSVInputFormat.DELIMITER,
+            CSVInputFormat.DELIMITER_DEFAULT);
+
+    return new CarbonStreamingRecordWriter<K, V>(
+            conf,
+            getDefaultWorkFile(job, null),
+            keyValueSeparator);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamingRecordWriter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamingRecordWriter.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamingRecordWriter.java
new file mode 100644
index 0000000..9d1951f
--- /dev/null
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/streaming/CarbonStreamingRecordWriter.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.hadoop.streaming;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.RecordWriter;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+
+public class CarbonStreamingRecordWriter<K,V> extends RecordWriter<K, V> {
+
+  private static final String utf8 = "UTF-8";
+
+  private static final byte[] newline;
+
+  static {
+
+    try {
+
+      newline = "\n".getBytes(utf8);
+
+    } catch (UnsupportedEncodingException uee) {
+
+      throw new IllegalArgumentException("Can't find " + utf8 + " encoding");
+    }
+  }
+
+  private FSDataOutputStream outputStream;
+
+  private FileSystem fs;
+
+  private Path file;
+
+  private volatile boolean isClosed;
+
+  private final byte[] keyValueSeparator;
+
+  public void initOut() throws IOException {
+
+    outputStream = fs.create(file, false);
+
+    isClosed = false;
+  }
+
+  public CarbonStreamingRecordWriter(
+          Configuration conf,
+          Path file,
+          String keyValueSeparator) throws IOException {
+
+    this.file = file;
+
+    fs = FileSystem.get(conf);
+
+    outputStream = fs.create(file, false);
+
+    isClosed = false;
+
+    try {
+
+      this.keyValueSeparator = keyValueSeparator.getBytes(utf8);
+
+    } catch (UnsupportedEncodingException uee) {
+
+      throw new IllegalArgumentException("Can't find " + utf8 + "encoding");
+
+    }
+
+  }
+
+  public CarbonStreamingRecordWriter(
+          Configuration conf,
+          Path file) throws IOException {
+
+    this(conf, file, ",");
+
+  }
+
+  /**
+   *  Write Object to byte stream.
+   */
+
+  private void writeObject(Object o) throws IOException {
+
+    if (o instanceof Text) {
+      Text to = (Text)o;
+
+      outputStream.write(to.getBytes(), 0, to.getLength());
+
+    } else {
+
+      outputStream.write(o.toString().getBytes(utf8));
+
+    }
+  }
+
+  /**
+   * Write streaming data as text file (temporary)
+   */
+
+  @Override
+  public synchronized void write(K key, V value) throws IOException {
+
+    boolean isNULLKey = key == null || key instanceof NullWritable;
+
+    boolean isNULLValue = value == null || value instanceof NullWritable;
+
+    if (isNULLKey && isNULLValue) {
+
+      return;
+    }
+
+    if (!isNULLKey) {
+
+      writeObject(key);
+    }
+
+    if (!isNULLKey || !isNULLValue) {
+
+      outputStream.write(keyValueSeparator);
+    }
+
+    if (!isNULLValue) {
+
+      writeObject(value);
+    }
+
+    outputStream.write(newline);
+  }
+
+  private void closeInternal() throws IOException {
+
+    if (!isClosed) {
+
+      outputStream.close();
+
+      isClosed = true;
+    }
+
+  }
+
+  public void flush() throws IOException {
+
+    outputStream.hflush();
+  }
+
+  public long getOffset() throws IOException {
+
+    return outputStream.getPos();
+  }
+
+  public void commit(boolean finalCommit) throws IOException {
+
+    closeInternal();
+
+    Path commitFile = new Path(file.getParent(),
+            CarbonTablePath.getCarbonDataPrefix() + System.currentTimeMillis());
+
+    fs.rename(file, commitFile);
+
+    if (!finalCommit) {
+      initOut();
+    }
+  }
+
+  @Override
+  public void close(TaskAttemptContext context) throws IOException, InterruptedException {
+
+    closeInternal();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/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 f4f8b75..d496de2 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
@@ -21,16 +21,19 @@ import scala.collection.JavaConverters._
 import scala.language.implicitConversions
 
 import org.apache.commons.lang.StringUtils
-import org.apache.hadoop.fs.Path
+import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hadoop.mapreduce.Job
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.execution.command.{TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory}
 import org.apache.spark.sql.execution.strategy.CarbonLateDecodeStrategy
 import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
 import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
 import org.apache.spark.sql.parser.CarbonSpark2SqlParser
 import org.apache.spark.sql.sources._
-import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.streaming.CarbonStreamingOutputWriterFactory
+import org.apache.spark.sql.types.{StringType, StructType}
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
@@ -41,12 +44,13 @@ import org.apache.carbondata.core.util.path.{CarbonStorePath, CarbonTablePath}
 import org.apache.carbondata.spark.CarbonOption
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 
+
 /**
  * Carbon relation provider compliant to data source api.
  * Creates carbon relations
  */
 class CarbonSource extends CreatableRelationProvider with RelationProvider
-  with SchemaRelationProvider with DataSourceRegister {
+  with SchemaRelationProvider with DataSourceRegister with FileFormat  {
 
   override def shortName(): String = "carbondata"
 
@@ -54,7 +58,7 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
   override def createRelation(sqlContext: SQLContext,
       parameters: Map[String, String]): BaseRelation = {
     CarbonEnv.getInstance(sqlContext.sparkSession)
-    // if path is provided we can directly create Hadoop relation. \
+    // if path is provided we can directly create Hadoop relation.
     // Otherwise create datasource relation
     parameters.get("tablePath") match {
       case Some(path) => CarbonDatasourceHadoopRelation(sqlContext.sparkSession,
@@ -178,7 +182,7 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
   /**
    * Returns the path of the table
    *
-   * @param sparkSession
+     * @param sparkSession
    * @param dbName
    * @param tableName
    * @return
@@ -203,11 +207,32 @@ class CarbonSource extends CreatableRelationProvider with RelationProvider
         (relation.tableMeta.tablePath, parameters)
       }
     } catch {
-      case ex: Exception =>
-        throw new Exception(s"Do not have $dbName and $tableName", ex)
+        case ex: Exception =>
+          throw new Exception(s"Do not have $dbName and $tableName", ex)
     }
   }
 
+  /**
+   * Prepares a write job and returns an [[OutputWriterFactory]].  Client side job preparation can
+   * be put here.  For example, user defined output committer can be configured here
+   * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass.
+   */
+  def prepareWrite(
+    sparkSession: SparkSession,
+    job: Job,
+    options: Map[String, String],
+    dataSchema: StructType): OutputWriterFactory = new CarbonStreamingOutputWriterFactory()
+
+  /**
+   * When possible, this method should return the schema of the given `files`.  When the format
+   * does not support inference, or no valid files are given should return None.  In these cases
+   * Spark will require that user specify the schema manually.
+   */
+  def inferSchema(
+    sparkSession: SparkSession,
+    options: Map[String, String],
+    files: Seq[FileStatus]): Option[StructType] = Some(new StructType().add("value", StringType))
+
 }
 
 object CarbonSource {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutpurWriteFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutpurWriteFactory.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutpurWriteFactory.scala
new file mode 100644
index 0000000..be69885
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutpurWriteFactory.scala
@@ -0,0 +1,88 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.streaming
+
+
+import java.util.concurrent.ConcurrentHashMap
+
+import org.apache.hadoop.mapreduce.TaskAttemptContext
+import org.apache.spark.sql.execution.datasources.OutputWriterFactory
+import org.apache.spark.sql.types.StructType
+
+import org.apache.carbondata.core.util.path.CarbonTablePath
+
+
+class CarbonStreamingOutputWriterFactory extends OutputWriterFactory {
+
+ /**
+  * When writing to a [[org.apache.spark.sql.execution.datasources.HadoopFsRelation]],
+  * this method gets called by each task on executor side
+  * to instantiate new [[org.apache.spark.sql.execution.datasources.OutputWriter]]s.
+  *
+  * @param path Path to write the file.
+  * @param dataSchema Schema of the rows to be written. Partition columns are not
+  *                   included in the schema if the relation being written is
+  *                   partitioned.
+  * @param context The Hadoop MapReduce task context.
+  */
+
+  override def newInstance(
+    path: String,
+
+    dataSchema: StructType,
+
+    context: TaskAttemptContext) : CarbonStreamingOutputWriter = {
+
+        new CarbonStreamingOutputWriter(path, context)
+  }
+
+  override def getFileExtension(context: TaskAttemptContext): String = {
+
+    CarbonTablePath.STREAM_FILE_NAME_EXT
+  }
+
+}
+
+object CarbonStreamingOutpurWriterFactory {
+
+  private[this] val writers = new ConcurrentHashMap[String, CarbonStreamingOutputWriter]()
+
+  def addWriter(path: String, writer: CarbonStreamingOutputWriter): Unit = {
+
+    if (writers.contains(path)) {
+      throw new IllegalArgumentException(path + "writer already exists")
+    }
+
+    writers.put(path, writer)
+  }
+
+  def getWriter(path: String): CarbonStreamingOutputWriter = {
+
+    writers.get(path)
+  }
+
+  def containsWriter(path: String): Boolean = {
+
+    writers.containsKey(path)
+  }
+
+  def removeWriter(path: String): Unit = {
+
+    writers.remove(path)
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/441907ee/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutputWriter.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutputWriter.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutputWriter.scala
new file mode 100644
index 0000000..dfc8ff3
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/streaming/CarbonStreamingOutputWriter.scala
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.streaming
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.{NullWritable, Text}
+import org.apache.hadoop.mapreduce.TaskAttemptContext
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.OutputWriter
+import org.apache.spark.sql.Row
+
+import org.apache.carbondata.hadoop.streaming.{CarbonStreamingOutputFormat, CarbonStreamingRecordWriter}
+
+class CarbonStreamingOutputWriter (
+    path: String,
+    context: TaskAttemptContext)
+    extends OutputWriter {
+
+  private[this] val buffer = new Text()
+
+  private val recordWriter: CarbonStreamingRecordWriter[NullWritable, Text] = {
+
+    val outputFormat = new CarbonStreamingOutputFormat[NullWritable, Text] () {
+
+      override def getDefaultWorkFile(context: TaskAttemptContext, extension: String) : Path = {
+        new Path(path)
+      }
+
+    /*
+     May need to override
+     def getOutputCommiter(c: TaskAttemptContext): OutputCommitter = {
+      null
+    }
+    */
+
+    }
+
+    outputFormat.
+      getRecordWriter(context).asInstanceOf[CarbonStreamingRecordWriter[NullWritable, Text]]
+  }
+
+  override def write(row: Row): Unit = {
+
+    throw new UnsupportedOperationException("call writeInternal")
+
+  }
+
+  override protected [sql] def writeInternal(row: InternalRow): Unit = {
+
+    val utf8string = row.getUTF8String(0)
+
+    buffer.set(utf8string.getBytes)
+
+    recordWriter.write(NullWritable.get(), buffer)
+
+  }
+
+  def getpath: String = path
+
+  override def close(): Unit = {
+
+    recordWriter.close(context)
+
+  }
+
+  def flush(): Unit = {
+
+    recordWriter.flush()
+
+  }
+
+  def getPos(): Long = {
+
+    recordWriter.getOffset()
+
+  }
+
+  def commit(finalCommit: Boolean): Unit = {
+
+    recordWriter.commit(finalCommit)
+
+  }
+}


[22/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/test/java/org/apache/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java b/processing/src/test/java/org/apache/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
deleted file mode 100644
index ad923a4..0000000
--- a/processing/src/test/java/org/apache/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.store.colgroup;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-import java.util.UUID;
-
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-
-/**
- * RowStore store min max test
- */
-public class ColGroupMinMaxTest {
-
-	/**
-	 * column groups
-	 */
-	int[][] columnGroups;
-
-	/**
-	 * surrogate key
-	 */
-	int[][] data;
-	/**
-	 * mdkey data
-	 */
-	byte[][] mdkeyData;
-
-	/**
-	 * min value of surrogates
-	 */
-	int[] min;
-	/**
-	 * max value of surrogates
-	 */
-	int[] max;
-	private ColGroupMinMax[] colGrpMinMax;
-	
-	private SegmentProperties segmentProperties;
-
-	@Before
-	public void setupBeforeClass() throws KeyGenException {
-		int[] dimLens = new int[] { 100000, 1000, 10, 100, 100, 10000, 1000, 10,
-				1000, 1 };
-		columnGroups = new int[][] { { 0, 1, 2 }, { 3, 4 }, { 5, 6 }, { 7, 8, 9 } };
-		segmentProperties = getSegmentProperties(dimLens, columnGroups);
-		initColGrpMinMax();
-		Random random = new Random();
-		data = new int[1000][];
-		min = new int[dimLens.length];
-		Arrays.fill(min, Integer.MAX_VALUE);
-		max = new int[dimLens.length];
-		Arrays.fill(max, Integer.MIN_VALUE);
-		for (int i = 0; i < 1000; i++) {
-
-			data[i] = new int[dimLens.length];
-			for (int j = 0; j < data[i].length; j++) {
-				data[i][j] = random.nextInt(dimLens[j]);
-			}
-			setMinData(data[i]);
-			setMaxData(data[i]);
-			System.out.println(Arrays.toString(data[i]));
-		}
-		mdkeyData = new byte[1000][];
-		for (int i = 0; i < 1000; i++) {
-			mdkeyData[i] = segmentProperties.getDimensionKeyGenerator().generateKey(data[i]);
-			evaluateColGrpMinMax(mdkeyData[i]);
-		}
-	}
-
-	private SegmentProperties getSegmentProperties(int[] dimLens, int[][] columnGroups) {
-		List<ColumnSchema> columnSchemas = new ArrayList<>();
-		for(int i=0;i<columnGroups.length;i++) {
-			  for(int j=0;j<columnGroups[i].length;j++) {
-			  	  columnSchemas.add(getDimensionColumn(i+j,i));
-			  }
-			
-		}
-		return new SegmentProperties(columnSchemas, dimLens);
-	}
-	private ColumnSchema getDimensionColumn(int var , int groupId) {
-  ColumnSchema dimColumn = new ColumnSchema();
-  dimColumn.setColumnar(false);
-  dimColumn.setColumnName("IMEI"+var);
-  dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
-  dimColumn.setDataType(DataType.STRING);
-  dimColumn.setDimensionColumn(true);
-  List<Encoding> encodeList =
-      new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  encodeList.add(Encoding.DICTIONARY);
-  dimColumn.setEncodingList(encodeList);
-  dimColumn.setColumnGroup(0);
-  dimColumn.setNumberOfChild(0);
-  return dimColumn;
-}
-
-	private void evaluateColGrpMinMax(byte[] mdkey) {
-
-		for (int colGrp = 0; colGrp < segmentProperties.getColumnGroups().length; colGrp++) {
-			if (segmentProperties.getColumnGroups()[colGrp].length > 0) {
-				colGrpMinMax[colGrp].add(mdkey);
-			}
-		}
-	}
-
-	private void initColGrpMinMax() {
-		int[][] colGrps = segmentProperties.getColumnGroups();
-		colGrpMinMax = new ColGroupMinMax[colGrps.length];
-		for (int colGrp = 0; colGrp < colGrps.length; colGrp++) {
-			if (colGrps[colGrp].length > 0) {
-				colGrpMinMax[colGrp] = new ColGroupMinMax(segmentProperties,
-						colGrp);
-			}
-		}
-	}
-
-	private void setMaxData(int[] data) {
-		for (int i = 0; i < max.length; i++) {
-			if (max[i] < data[i]) {
-				max[i] = data[i];
-			}
-		}
-
-	}
-
-	private void setMinData(int[] data) {
-		for (int i = 0; i < min.length; i++) {
-			if (min[i] > data[i]) {
-				min[i] = data[i];
-			}
-		}
-	}
-
-	private void assertMinMax(byte[] min, byte[] max, int[] columnGroup)
-			throws KeyGenException {
-
-		int columnStartIndex = 0;
-		for (int i = 0; i < columnGroup.length; i++) {
-			int col = columnGroup[i];
-			int[] maskByteRange = getMaskByteRange(col);
-			int[] maskBytePosition = new int[segmentProperties.getDimensionKeyGenerator().getKeySizeInBytes()];
-			updateMaskedKeyRanges(maskBytePosition, maskByteRange);
-
-			byte[] columnMin = new byte[maskByteRange.length];
-			System.arraycopy(min, columnStartIndex, columnMin, 0, maskByteRange.length);
-			byte[] columnMax = new byte[maskByteRange.length];
-			System.arraycopy(max, columnStartIndex, columnMax, 0, maskByteRange.length);
-
-			long[] minKeyArray = segmentProperties.getDimensionKeyGenerator().getKeyArray(columnMin, maskBytePosition);
-			long[] maxKeyArray = segmentProperties.getDimensionKeyGenerator().getKeyArray(columnMax, maskBytePosition);
-			System.out.println("calculated:(min,max) for column " + col + ":("
-					+ minKeyArray[col] + "," + maxKeyArray[col] + ")");
-			System.out.println("correct:(min,max) for column " + col + ":("
-					+ this.min[col] + "," + this.max[col] + ")");
-			columnStartIndex += maskByteRange.length;
-			Assert.assertEquals(minKeyArray[col], this.min[col]);
-			Assert.assertEquals(maxKeyArray[col], this.max[col]);
-
-		}
-
-	}
-
-	private DataHolder[] getDataHolders(int noOfColumn, int noOfRow) {
-		DataHolder[] dataHolders = new DataHolder[noOfColumn];
-		for (int colGrp = 0; colGrp < noOfColumn; colGrp++) {
-			if (segmentProperties.getColumnGroupModel().isColumnar(colGrp)) {
-				dataHolders[colGrp] = new ColumnDataHolder(noOfRow);
-			} else {
-				dataHolders[colGrp] = new ColGroupDataHolder(
-						segmentProperties.getFixedLengthKeySplitter().getBlockKeySize()[colGrp], noOfRow,
-						colGrpMinMax[colGrp]);
-			}
-		}
-		return dataHolders;
-	}
-
-	private int[] getMaskByteRange(int col) {
-		Set<Integer> integers = new HashSet<>();
-		int[] range = segmentProperties.getDimensionKeyGenerator().getKeyByteOffsets(col);
-		for (int j = range[0]; j <= range[1]; j++) {
-			integers.add(j);
-		}
-		int[] byteIndexs = new int[integers.size()];
-		int j = 0;
-		for (Iterator<Integer> iterator = integers.iterator(); iterator.hasNext();) {
-			Integer integer = (Integer) iterator.next();
-			byteIndexs[j++] = integer.intValue();
-		}
-		return byteIndexs;
-	}
-
-	private void updateMaskedKeyRanges(int[] maskedKey, int[] maskedKeyRanges) {
-		Arrays.fill(maskedKey, -1);
-		for (int i = 0; i < maskedKeyRanges.length; i++) {
-			maskedKey[maskedKeyRanges[i]] = i;
-		}
-	}
-}


[23/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 e0d4b73..70a8703 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
@@ -50,7 +50,7 @@ import org.apache.carbondata.processing.store.writer.CarbonDataWriterVo;
  * <Column3 Data ChunkV3><Column3<Page1><Page2><Page3><Page4>>
  * <Column4 Data ChunkV3><Column4<Page1><Page2><Page3><Page4>>
  */
-public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter<short[]> {
+public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(CarbonFactDataWriterImplV3.class.getName());

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/BadRecordsLogger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/BadRecordsLogger.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/BadRecordsLogger.java
deleted file mode 100644
index b93fcb7..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/BadRecordsLogger.java
+++ /dev/null
@@ -1,278 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.surrogatekeysgenerator.csvbased;
-
-import java.io.BufferedWriter;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.nio.charset.Charset;
-import java.util.HashMap;
-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.impl.FileFactory;
-import org.apache.carbondata.core.datastore.impl.FileFactory.FileType;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-
-public class BadRecordsLogger {
-
-  /**
-   * Comment for <code>LOGGER</code>
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(BadRecordsLogger.class.getName());
-  /**
-   * Which holds the key and if any bad rec found to check from API to update
-   * the status
-   */
-  private static Map<String, String> badRecordEntry =
-      new HashMap<String, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  /**
-   * File Name
-   */
-  private String fileName;
-  /**
-   * Store path
-   */
-  private String storePath;
-  /**
-   * FileChannel
-   */
-  private BufferedWriter bufferedWriter;
-  private DataOutputStream outStream;
-  /**
-   * csv file writer
-   */
-  private BufferedWriter bufferedCSVWriter;
-  private DataOutputStream outCSVStream;
-  /**
-   * bad record log file path
-   */
-  private String logFilePath;
-  /**
-   * csv file path
-   */
-  private String csvFilePath;
-
-  /**
-   * task key which is DatabaseName/TableName/tablename
-   */
-  private String taskKey;
-
-  private boolean badRecordsLogRedirect;
-
-  private boolean badRecordLoggerEnable;
-
-  private boolean badRecordConvertNullDisable;
-
-  private boolean isDataLoadFail;
-
-  // private final Object syncObject =new Object();
-
-  public BadRecordsLogger(String key, String fileName, String storePath,
-      boolean badRecordsLogRedirect, boolean badRecordLoggerEnable,
-      boolean badRecordConvertNullDisable, boolean isDataLoadFail) {
-    // Initially no bad rec
-    taskKey = key;
-    this.fileName = fileName;
-    this.storePath = storePath;
-    this.badRecordsLogRedirect = badRecordsLogRedirect;
-    this.badRecordLoggerEnable = badRecordLoggerEnable;
-    this.badRecordConvertNullDisable = badRecordConvertNullDisable;
-    this.isDataLoadFail = isDataLoadFail;
-  }
-
-  /**
-   * @param key DatabaseNaame/TableName/tablename
-   * @return return "Partially"
-   */
-  public static String hasBadRecord(String key) {
-    return badRecordEntry.get(key);
-  }
-
-  /**
-   * @param key DatabaseNaame/TableName/tablename
-   * @return remove key from the map
-   */
-  public static String removeBadRecordKey(String key) {
-    return badRecordEntry.remove(key);
-  }
-
-  public void addBadRecordsToBuilder(Object[] row, String reason)
-      throws CarbonDataLoadingException {
-    if (badRecordsLogRedirect || badRecordLoggerEnable) {
-      StringBuilder logStrings = new StringBuilder();
-      int size = row.length;
-      int count = size;
-      for (int i = 0; i < size; i++) {
-        if (null == row[i]) {
-          char ch =
-              logStrings.length() > 0 ? logStrings.charAt(logStrings.length() - 1) : (char) -1;
-          if (ch == ',') {
-            logStrings = logStrings.deleteCharAt(logStrings.lastIndexOf(","));
-          }
-          break;
-        } else if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(row[i].toString())) {
-          logStrings.append("null");
-        } else {
-          logStrings.append(row[i]);
-        }
-        if (count > 1) {
-          logStrings.append(',');
-        }
-        count--;
-      }
-      if (badRecordsLogRedirect) {
-        writeBadRecordsToCSVFile(logStrings);
-      }
-      if (badRecordLoggerEnable) {
-        logStrings.append("----->");
-        if (null != reason) {
-          if (reason.indexOf(CarbonCommonConstants.MEMBER_DEFAULT_VAL) > -1) {
-            logStrings
-                .append(reason.replace(CarbonCommonConstants.MEMBER_DEFAULT_VAL, "null"));
-          } else {
-            logStrings.append(reason);
-          }
-        }
-        writeBadRecordsToFile(logStrings);
-      }
-    } else {
-      // setting partial success entry since even if bad records are there then load
-      // status should be partial success regardless of bad record logged
-      badRecordEntry.put(taskKey, "Partially");
-    }
-  }
-
-  /**
-   *
-   */
-  private synchronized void writeBadRecordsToFile(StringBuilder logStrings)
-      throws CarbonDataLoadingException {
-    if (null == logFilePath) {
-      logFilePath =
-          this.storePath + File.separator + this.fileName + CarbonCommonConstants.LOG_FILE_EXTENSION
-              + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-    }
-    try {
-      if (null == bufferedWriter) {
-        FileType fileType = FileFactory.getFileType(storePath);
-        if (!FileFactory.isFileExist(this.storePath, fileType)) {
-          // create the folders if not exist
-          FileFactory.mkdirs(this.storePath, fileType);
-
-          // create the files
-          FileFactory.createNewFile(logFilePath, fileType);
-        }
-
-        outStream = FileFactory.getDataOutputStream(logFilePath, fileType);
-
-        bufferedWriter = new BufferedWriter(new OutputStreamWriter(outStream,
-            Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-
-      }
-      bufferedWriter.write(logStrings.toString());
-      bufferedWriter.newLine();
-    } catch (FileNotFoundException e) {
-      LOGGER.error("Bad Log Files not found");
-      throw new CarbonDataLoadingException("Bad Log Files not found", e);
-    } catch (IOException e) {
-      LOGGER.error("Error While writing bad record log File");
-      throw new CarbonDataLoadingException("Error While writing bad record log File", e);
-    } finally {
-      // if the Bad record file is created means it partially success
-      // if any entry present with key that means its have bad record for
-      // that key
-      badRecordEntry.put(taskKey, "Partially");
-    }
-  }
-
-  /**
-   * method will write the row having bad record in the csv file.
-   *
-   * @param logStrings
-   */
-  private synchronized void writeBadRecordsToCSVFile(StringBuilder logStrings)
-      throws CarbonDataLoadingException {
-    if (null == csvFilePath) {
-      csvFilePath =
-          this.storePath + File.separator + this.fileName + CarbonCommonConstants.CSV_FILE_EXTENSION
-              + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-    }
-    try {
-      if (null == bufferedCSVWriter) {
-        FileType fileType = FileFactory.getFileType(storePath);
-        if (!FileFactory.isFileExist(this.storePath, fileType)) {
-          // create the folders if not exist
-          FileFactory.mkdirs(this.storePath, fileType);
-
-          // create the files
-          FileFactory.createNewFile(csvFilePath, fileType);
-        }
-
-        outCSVStream = FileFactory.getDataOutputStream(csvFilePath, fileType);
-
-        bufferedCSVWriter = new BufferedWriter(new OutputStreamWriter(outCSVStream,
-            Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-
-      }
-      bufferedCSVWriter.write(logStrings.toString());
-      bufferedCSVWriter.newLine();
-    } catch (FileNotFoundException e) {
-      LOGGER.error("Bad record csv Files not found");
-      throw new CarbonDataLoadingException("Bad record csv Files not found", e);
-    } catch (IOException e) {
-      LOGGER.error("Error While writing bad record csv File");
-      throw new CarbonDataLoadingException("Error While writing bad record csv File", e);
-    }
-    finally {
-      badRecordEntry.put(taskKey, "Partially");
-    }
-  }
-
-  public boolean isBadRecordConvertNullDisable() {
-    return badRecordConvertNullDisable;
-  }
-
-  public boolean isDataLoadFail() {
-    return isDataLoadFail;
-  }
-
-  public boolean isBadRecordLoggerEnable() {
-    return badRecordLoggerEnable;
-  }
-
-  public boolean isBadRecordsLogRedirect() {
-    return badRecordsLogRedirect;
-  }
-
-  /**
-   * closeStreams void
-   */
-  public synchronized void closeStreams() {
-    CarbonUtil.closeStreams(bufferedWriter, outStream, bufferedCSVWriter, outCSVStream);
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 fabb5a5..79e49ef 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
@@ -52,10 +52,10 @@ import org.apache.carbondata.processing.datatypes.ArrayDataType;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
 import org.apache.carbondata.processing.datatypes.PrimitiveDataType;
 import org.apache.carbondata.processing.datatypes.StructDataType;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
-import org.apache.carbondata.processing.newflow.DataField;
-import org.apache.carbondata.processing.newflow.sort.SortScopeOptions;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.model.CarbonDataLoadSchema;
+import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
 
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.StringUtils;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..8681269
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -0,0 +1,890 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.File;
+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.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.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.block.Distributable;
+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.datastore.row.LoadStatusType;
+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.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;
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+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.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.merger.NodeBlockRelation;
+import org.apache.carbondata.processing.merger.NodeMultiBlockRelation;
+
+import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
+
+public final class CarbonLoaderUtil {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CarbonLoaderUtil.class.getName());
+
+  private CarbonLoaderUtil() {
+  }
+
+  public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
+    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
+    CarbonTablePath carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
+
+    for (int i = 0; i < carbonTable.getPartitionCount(); i++) {
+      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(i + "", currentLoad + "");
+      deleteStorePath(segmentPath);
+    }
+  }
+
+  /**
+   * the method returns true if the segment has carbondata file else returns false.
+   *
+   * @param loadModel
+   * @param currentLoad
+   * @return
+   */
+  public static boolean isValidSegment(CarbonLoadModel loadModel,
+      int currentLoad) {
+    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema()
+        .getCarbonTable();
+    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(
+        loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
+
+    int fileCount = 0;
+    int partitionCount = carbonTable.getPartitionCount();
+    for (int i = 0; i < partitionCount; i++) {
+      String segmentPath = carbonTablePath.getCarbonDataDirectoryPath(i + "",
+          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;
+    }
+    return true;
+  }
+  public static void deletePartialLoadDataIfExist(CarbonLoadModel loadModel,
+      final boolean isCompactionFlow) throws IOException {
+    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
+    String metaDataLocation = carbonTable.getMetaDataFilepath();
+    final LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metaDataLocation);
+    CarbonTablePath carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
+
+    //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);
+      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) && details[j].getPartitionCount()
+                  .equals(partitionCount)) {
+                found = true;
+                break;
+              }
+            }
+            return !found;
+          }
+        });
+        for (int k = 0; k < listFiles.length; k++) {
+          String segmentId =
+              CarbonTablePath.DataPathUtil.getSegmentId(listFiles[k].getAbsolutePath() + "/dummy");
+          if (isCompactionFlow) {
+            if (segmentId.contains(".")) {
+              deleteStorePath(listFiles[k].getAbsolutePath());
+            }
+          } else {
+            if (!segmentId.contains(".")) {
+              deleteStorePath(listFiles[k].getAbsolutePath());
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private static void deleteStorePath(String path) {
+    try {
+      FileType fileType = FileFactory.getFileType(path);
+      if (FileFactory.isFileExist(path, fileType)) {
+        CarbonFile carbonFile = FileFactory.getCarbonFile(path, fileType);
+        CarbonUtil.deleteFoldersAndFiles(carbonFile);
+      }
+    } catch (IOException | InterruptedException e) {
+      LOGGER.error("Unable to delete the given path :: " + e.getMessage());
+    }
+  }
+
+
+  /**
+   * This method will delete the local data load folder location after data load is complete
+   *
+   * @param loadModel
+   */
+  public static void deleteLocalDataLoadFolderLocation(CarbonLoadModel loadModel,
+      boolean isCompactionFlow, boolean isAltPartitionFlow) {
+    String databaseName = loadModel.getDatabaseName();
+    String tableName = loadModel.getTableName();
+    String tempLocationKey = CarbonDataProcessorUtil
+        .getTempStoreLocationKey(databaseName, tableName, loadModel.getSegmentId(),
+            loadModel.getTaskNo(), isCompactionFlow, isAltPartitionFlow);
+    // form local store location
+    final String localStoreLocations = CarbonProperties.getInstance().getProperty(tempLocationKey);
+    if (localStoreLocations == null) {
+      throw new RuntimeException("Store location not set for the key " + tempLocationKey);
+    }
+    // submit local folder clean up in another thread so that main thread execution is not blocked
+    ExecutorService localFolderDeletionService = Executors.newFixedThreadPool(1);
+    try {
+      localFolderDeletionService.submit(new Callable<Void>() {
+        @Override public Void call() throws Exception {
+          long startTime = System.currentTimeMillis();
+          String[] locArray = StringUtils.split(localStoreLocations, File.pathSeparator);
+          for (String loc : locArray) {
+            try {
+              CarbonUtil.deleteFoldersAndFiles(new File(loc));
+            } catch (IOException | InterruptedException e) {
+              LOGGER.error(e,
+                  "Failed to delete local data load folder location: " + loc);
+            }
+          }
+          LOGGER.info("Deleted the local store location: " + localStoreLocations
+                + " : Time taken: " + (System.currentTimeMillis() - startTime));
+          return null;
+        }
+      });
+    } finally {
+      if (null != localFolderDeletionService) {
+        localFolderDeletionService.shutdown();
+      }
+    }
+
+  }
+
+  /**
+   * This API will write the load level metadata for the loadmanagement module inorder to
+   * manage the load and query execution management smoothly.
+   *
+   * @param newMetaEntry
+   * @param loadModel
+   * @return boolean which determines whether status update is done or not.
+   * @throws IOException
+   */
+  public static boolean recordLoadMetadata(LoadMetadataDetails newMetaEntry,
+      CarbonLoadModel loadModel, boolean loadStartEntry, boolean insertOverwrite)
+      throws IOException, InterruptedException {
+    boolean status = false;
+    String metaDataFilepath =
+        loadModel.getCarbonDataLoadSchema().getCarbonTable().getMetaDataFilepath();
+    AbsoluteTableIdentifier absoluteTableIdentifier =
+        loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
+    CarbonTablePath carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            absoluteTableIdentifier.getCarbonTableIdentifier());
+    String tableStatusPath = carbonTablePath.getTableStatusFilePath();
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+    try {
+      if (carbonLock.lockWithRetries()) {
+        LOGGER.info(
+            "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
+                + " for table status updation");
+        LoadMetadataDetails[] listOfLoadFolderDetailsArray =
+            SegmentStatusManager.readLoadMetadata(metaDataFilepath);
+        List<LoadMetadataDetails> listOfLoadFolderDetails =
+            new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        List<CarbonFile> staleFolders = new ArrayList<>();
+        Collections.addAll(listOfLoadFolderDetails, listOfLoadFolderDetailsArray);
+        // create a new segment Id if load has just begun else add the already generated Id
+        if (loadStartEntry) {
+          String segmentId =
+              String.valueOf(SegmentStatusManager.createNewSegmentId(listOfLoadFolderDetailsArray));
+          newMetaEntry.setLoadName(segmentId);
+          loadModel.setLoadMetadataDetails(listOfLoadFolderDetails);
+          loadModel.setSegmentId(segmentId);
+          // Exception should be thrown if:
+          // 1. If insert overwrite is in progress and any other load or insert operation
+          // is triggered
+          // 2. If load or insert into operation is in progress and insert overwrite operation
+          // is triggered
+          for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
+            if (entry.getLoadStatus().equals(LoadStatusType.INSERT_OVERWRITE.getMessage())) {
+              throw new RuntimeException("Already insert overwrite is in progress");
+            } else if (
+                newMetaEntry.getLoadStatus().equals(LoadStatusType.INSERT_OVERWRITE.getMessage())
+                    && entry.getLoadStatus().equals(LoadStatusType.IN_PROGRESS.getMessage())) {
+              throw new RuntimeException("Already insert into or load is in progress");
+            }
+          }
+          listOfLoadFolderDetails.add(newMetaEntry);
+        } else {
+          newMetaEntry.setLoadName(String.valueOf(loadModel.getSegmentId()));
+          // existing entry needs to be overwritten as the entry will exist with some
+          // intermediate status
+          int indexToOverwriteNewMetaEntry = 0;
+          for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
+            if (entry.getLoadName().equals(newMetaEntry.getLoadName())
+                && entry.getLoadStartTime() == newMetaEntry.getLoadStartTime()) {
+              break;
+            }
+            indexToOverwriteNewMetaEntry++;
+          }
+          if (listOfLoadFolderDetails.get(indexToOverwriteNewMetaEntry).getLoadStatus()
+              .equals(CarbonCommonConstants.MARKED_FOR_DELETE)) {
+            throw new RuntimeException("It seems insert overwrite has been issued during load");
+          }
+          if (insertOverwrite) {
+            for (LoadMetadataDetails entry : listOfLoadFolderDetails) {
+              if (!entry.getLoadStatus().equals(LoadStatusType.INSERT_OVERWRITE.getMessage())) {
+                entry.setLoadStatus(CarbonCommonConstants.MARKED_FOR_DELETE);
+                // For insert overwrite, we will delete the old segment folder immediately
+                // So collect the old segments here
+                String path = carbonTablePath.getCarbonDataDirectoryPath("0", 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))) {
+                  staleFolders.add(FileFactory.getCarbonFile(path));
+                }
+              }
+            }
+          }
+          listOfLoadFolderDetails.set(indexToOverwriteNewMetaEntry, newMetaEntry);
+        }
+        SegmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetails
+            .toArray(new LoadMetadataDetails[listOfLoadFolderDetails.size()]));
+        // Delete all old stale segment folders
+        for (CarbonFile staleFolder : staleFolders) {
+          // try block is inside for loop because even if there is failure in deletion of 1 stale
+          // folder still remaining stale folders should be deleted
+          try {
+            CarbonUtil.deleteFoldersAndFiles(staleFolder);
+          } catch (IOException | InterruptedException e) {
+            LOGGER.error("Failed to delete stale folder: " + e.getMessage());
+          }
+        }
+        status = true;
+      } else {
+        LOGGER.error("Not able to acquire the lock for Table status updation for table " + loadModel
+            .getDatabaseName() + "." + loadModel.getTableName());
+      };
+    } finally {
+      if (carbonLock.unlock()) {
+        LOGGER.info(
+            "Table unlocked successfully after table status updation" + loadModel.getDatabaseName()
+                + "." + loadModel.getTableName());
+      } else {
+        LOGGER.error(
+            "Unable to unlock Table lock for table" + loadModel.getDatabaseName() + "." + loadModel
+                .getTableName() + " during table status updation");
+      }
+    }
+    return status;
+  }
+
+  /**
+   * Method to create new entry for load in table status file
+   *
+   * @param loadMetadataDetails
+   * @param loadStatus
+   * @param loadStartTime
+   * @param addLoadEndTime
+   */
+  public static void populateNewLoadMetaEntry(LoadMetadataDetails loadMetadataDetails,
+      String loadStatus, long loadStartTime, boolean addLoadEndTime) {
+    if (addLoadEndTime) {
+      long loadEndDate = CarbonUpdateUtil.readCurrentTime();
+      loadMetadataDetails.setLoadEndTime(loadEndDate);
+    }
+    loadMetadataDetails.setLoadStatus(loadStatus);
+    loadMetadataDetails.setLoadStartTime(loadStartTime);
+  }
+
+  public static void writeLoadMetadata(String storeLocation, String dbName, String tableName,
+      List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
+    CarbonTablePath carbonTablePath =
+        CarbonStorePath.getCarbonTablePath(storeLocation, dbName, tableName);
+    String dataLoadLocation = carbonTablePath.getTableStatusFilePath();
+
+    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 Dictionary getDictionary(DictionaryColumnUniqueIdentifier columnIdentifier,
+      String carbonStorePath) throws IOException {
+    Cache<DictionaryColumnUniqueIdentifier, Dictionary> dictCache =
+        CacheProvider.getInstance().createCache(CacheType.REVERSE_DICTIONARY, carbonStorePath);
+    return dictCache.get(columnIdentifier);
+  }
+
+  public static Dictionary getDictionary(CarbonTableIdentifier tableIdentifier,
+      ColumnIdentifier columnIdentifier, String carbonStorePath, DataType dataType)
+      throws IOException {
+    return getDictionary(
+        new DictionaryColumnUniqueIdentifier(tableIdentifier, columnIdentifier, dataType,
+            CarbonStorePath.getCarbonTablePath(carbonStorePath, tableIdentifier)),
+        carbonStorePath);
+  }
+
+  /**
+   * This method will divide the blocks among the tasks of the nodes as per the data locality
+   *
+   * @param blockInfos
+   * @param noOfNodesInput -1 if number of nodes has to be decided
+   *                       based on block location information
+   * @param parallelism    total no of tasks to execute in parallel
+   * @return
+   */
+  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);
+    int taskPerNode = parallelism / mapOfNodes.size();
+    //assigning non zero value to noOfTasksPerNode
+    int noOfTasksPerNode = taskPerNode == 0 ? 1 : taskPerNode;
+    // divide the blocks of a node among the tasks of the node.
+    return assignBlocksToTasksPerNode(mapOfNodes, noOfTasksPerNode);
+  }
+
+  /**
+   * 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,
+      int noOfNodesInput) {
+    return nodeBlockMapping(blockInfos, noOfNodesInput, null);
+  }
+
+  /**
+   * 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
+   * @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 noOfNodesInput -1 if number of nodes has to be decided
+   *                       based on block location information
+   * @return
+   */
+  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;
+
+    // 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);
+
+    // 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(nodeBlocksMap, uniqueBlocks, blocksPerNode, activeNodes);
+
+    return nodeBlocksMap;
+  }
+
+  /**
+   * Assigning the blocks of a node to tasks.
+   *
+   * @param nodeBlocksMap nodeName to list of blocks mapping
+   * @param noOfTasksPerNode
+   * @return
+   */
+  private static Map<String, List<List<Distributable>>> assignBlocksToTasksPerNode(
+      Map<String, List<Distributable>> nodeBlocksMap, int noOfTasksPerNode) {
+    Map<String, List<List<Distributable>>> outputMap =
+        new HashMap<String, List<List<Distributable>>>(
+            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+    // for each node
+    for (Map.Entry<String, List<Distributable>> eachNode : nodeBlocksMap.entrySet()) {
+
+      List<Distributable> blockOfEachNode = eachNode.getValue();
+      //sorting the block so same block will be give to same executor
+      Collections.sort(blockOfEachNode);
+      // create the task list for each node.
+      createTaskListForNode(outputMap, noOfTasksPerNode, eachNode.getKey());
+
+      // take all the block of node and divide it among the tasks of a node.
+      divideBlockToTasks(outputMap, eachNode.getKey(), blockOfEachNode);
+    }
+
+    return outputMap;
+  }
+
+  /**
+   * This will divide the blocks of a node to tasks of the node.
+   *
+   * @param outputMap
+   * @param key
+   * @param blockOfEachNode
+   */
+  private static void divideBlockToTasks(Map<String, List<List<Distributable>>> outputMap,
+      String key, List<Distributable> blockOfEachNode) {
+
+    List<List<Distributable>> taskLists = outputMap.get(key);
+    int tasksOfNode = taskLists.size();
+    int i = 0;
+    for (Distributable block : blockOfEachNode) {
+
+      taskLists.get(i % tasksOfNode).add(block);
+      i++;
+    }
+
+  }
+
+  /**
+   * This will create the empty list for each task of a node.
+   *
+   * @param outputMap
+   * @param noOfTasksPerNode
+   * @param key
+   */
+  private static void createTaskListForNode(Map<String, List<List<Distributable>>> outputMap,
+      int noOfTasksPerNode, String key) {
+    List<List<Distributable>> nodeTaskList =
+        new ArrayList<List<Distributable>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    for (int i = 0; i < noOfTasksPerNode; i++) {
+      List<Distributable> eachTask =
+          new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+      nodeTaskList.add(eachTask);
+
+    }
+    outputMap.put(key, nodeTaskList);
+
+  }
+
+  /**
+   * 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(Map<String, List<Distributable>> outputMap,
+      Set<Distributable> uniqueBlocks, int noOfBlocksPerNode, List<String> activeNodes) {
+
+    if (activeNodes != null) {
+      for (String activeNode : activeNodes) {
+        List<Distributable> blockLst = outputMap.get(activeNode);
+        if (null == blockLst) {
+          blockLst = new ArrayList<Distributable>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        }
+        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
+        if (blockLst.size() > 0) {
+          outputMap.put(activeNode, blockLst);
+        }
+      }
+    } else {
+      for (Map.Entry<String, List<Distributable>> entry : outputMap.entrySet()) {
+        List<Distributable> blockLst = entry.getValue();
+        populateBlocks(uniqueBlocks, noOfBlocksPerNode, blockLst);
+      }
+
+    }
+
+    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();
+      }
+    }
+  }
+
+  /**
+   * The method populate the blockLst to be allocate to a specific node.
+   * @param uniqueBlocks
+   * @param noOfBlocksPerNode
+   * @param blockLst
+   */
+  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();
+      blockLst.add(block);
+      blocks.remove();
+      if (blockLst.size() >= noOfBlocksPerNode) {
+        break;
+      }
+    }
+  }
+
+  /**
+   * To create the final output of the Node and Data blocks
+   *
+   * @param outputMap
+   * @param blocksPerNode
+   * @param uniqueBlocks
+   * @param nodeAndBlockMapping
+   * @param activeNodes
+   */
+  private static void createOutputMap(Map<String, List<Distributable>> outputMap, int blocksPerNode,
+      Set<Distributable> uniqueBlocks, Map<String, List<Distributable>> nodeAndBlockMapping,
+      List<String> activeNodes) {
+
+    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);
+
+    for (NodeMultiBlockRelation nodeMultiBlockRelation : multiBlockRelations) {
+      String nodeName = nodeMultiBlockRelation.getNode();
+      //assign the block to the node only if the node is active
+      String activeExecutor = nodeName;
+      if (null != activeNodes) {
+        activeExecutor = getActiveExecutor(activeNodes, nodeName);
+        if (null == activeExecutor) {
+          continue;
+        }
+      }
+      // this loop will be for each NODE
+      int nodeCapacity = 0;
+      // loop thru blocks of each Node
+      for (Distributable block : nodeMultiBlockRelation.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++;
+            uniqueBlocks.remove(block);
+          } else {
+            // No need to continue loop as node is full
+            break;
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * method validates whether the node is active or not.
+   *
+   * @param activeNode
+   * @param nodeName
+   * @return returns true if active else false.
+   */
+  private static String getActiveExecutor(List activeNode, String nodeName) {
+    boolean isActiveNode = activeNode.contains(nodeName);
+    if (isActiveNode) {
+      return nodeName;
+    }
+    //if localhost then retrieve the localhost name then do the check
+    else if (nodeName.equals("localhost")) {
+      try {
+        String hostName = InetAddress.getLocalHost().getHostName();
+        isActiveNode = activeNode.contains(hostName);
+        if (isActiveNode) {
+          return hostName;
+        }
+      } catch (UnknownHostException ue) {
+        isActiveNode = false;
+      }
+    } else {
+      try {
+        String hostAddress = InetAddress.getByName(nodeName).getHostAddress();
+        isActiveNode = activeNode.contains(hostAddress);
+        if (isActiveNode) {
+          return hostAddress;
+        }
+      } catch (UnknownHostException ue) {
+        isActiveNode = false;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * Create the Node and its related blocks Mapping and put in a Map
+   *
+   * @param flattenedList
+   * @param nodeAndBlockMapping
+   */
+  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 (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);
+      }
+    }
+  }
+
+  /**
+   * This method will get the store location for the given path, segment id and partition id
+   *
+   * @param carbonStorePath
+   * @param segmentId
+   */
+  public static void checkAndCreateCarbonDataLocation(String carbonStorePath,
+      String segmentId, CarbonTable carbonTable) {
+    CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
+    CarbonTablePath carbonTablePath =
+        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
+    String carbonDataDirectoryPath =
+        carbonTablePath.getCarbonDataDirectoryPath("0", segmentId);
+    CarbonUtil.checkAndCreateFolder(carbonDataDirectoryPath);
+  }
+
+  /**
+   * 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;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..ec91472
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonQueryUtil.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.Collections;
+import java.util.HashMap;
+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.processing.partition.Partition;
+import org.apache.carbondata.processing.partition.impl.DefaultLoadBalancer;
+import org.apache.carbondata.processing.partition.impl.PartitionMultiFileImpl;
+import org.apache.carbondata.processing.partition.impl.QueryPartitionHelper;
+import org.apache.carbondata.processing.splits.TableSplit;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * This utilty parses the Carbon query plan to actual query model object.
+ */
+public class CarbonQueryUtil {
+
+  private CarbonQueryUtil() {
+
+  }
+
+  /**
+   * It creates the one split for each region server.
+   */
+  public static synchronized TableSplit[] getTableSplits(String databaseName, String tableName,
+      CarbonQueryPlan queryPlan) {
+
+    //Just create splits depends on locations of region servers
+    List<Partition> allPartitions = null;
+    if (queryPlan == null) {
+      allPartitions =
+          QueryPartitionHelper.getInstance().getAllPartitions(databaseName, tableName);
+    } else {
+      allPartitions =
+          QueryPartitionHelper.getInstance().getPartitionsForQuery(queryPlan);
+    }
+    TableSplit[] splits = new TableSplit[allPartitions.size()];
+    for (int i = 0; i < splits.length; i++) {
+      splits[i] = new TableSplit();
+      List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      Partition partition = allPartitions.get(i);
+      String location = QueryPartitionHelper.getInstance()
+          .getLocation(partition, databaseName, tableName);
+      locations.add(location);
+      splits[i].setPartition(partition);
+      splits[i].setLocations(locations);
+    }
+
+    return splits;
+  }
+
+  /**
+   * It creates the one split for each region server.
+   */
+  public static TableSplit[] getTableSplitsForDirectLoad(String sourcePath) {
+
+    //Just create splits depends on locations of region servers
+    DefaultLoadBalancer loadBalancer = null;
+    List<Partition> allPartitions = getAllFilesForDataLoad(sourcePath);
+    loadBalancer = new DefaultLoadBalancer(new ArrayList<String>(), allPartitions);
+    TableSplit[] tblSplits = new TableSplit[allPartitions.size()];
+    for (int i = 0; i < tblSplits.length; i++) {
+      tblSplits[i] = new TableSplit();
+      List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+      Partition partition = allPartitions.get(i);
+      String location = loadBalancer.getNodeForPartitions(partition);
+      locations.add(location);
+      tblSplits[i].setPartition(partition);
+      tblSplits[i].setLocations(locations);
+    }
+    return tblSplits;
+  }
+
+  /**
+   * split sourcePath by comma
+   */
+  public static void splitFilePath(String sourcePath, List<String> partitionsFiles,
+      String separator) {
+    if (StringUtils.isNotEmpty(sourcePath)) {
+      String[] files = sourcePath.split(separator);
+      Collections.addAll(partitionsFiles, files);
+    }
+  }
+
+  private static List<Partition> getAllFilesForDataLoad(String sourcePath) {
+    List<String> files = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    splitFilePath(sourcePath, files, CarbonCommonConstants.COMMA);
+    List<Partition> partitionList =
+        new ArrayList<Partition>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    Map<Integer, List<String>> partitionFiles = new HashMap<Integer, List<String>>();
+
+    partitionFiles.put(0, new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN));
+    partitionList.add(new PartitionMultiFileImpl(0 + "", partitionFiles.get(0)));
+
+    for (int i = 0; i < files.size(); i++) {
+      partitionFiles.get(0).add(files.get(i));
+    }
+    return partitionList;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..d668cc2
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/DeleteLoadFolders.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 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.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
+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 dbName
+   * @param tableName
+   * @param storeLocation
+   * @param partitionId
+   * @param oneLoad
+   * @return
+   */
+  private static String getSegmentPath(String dbName, String tableName, String storeLocation,
+      int partitionId, LoadMetadataDetails oneLoad) {
+    CarbonTablePath carbon = new CarbonStorePath(storeLocation).getCarbonTablePath(
+        new CarbonTableIdentifier(dbName, tableName, ""));
+    String segmentId = oneLoad.getLoadName();
+    return carbon.getCarbonDataDirectoryPath("" + partitionId, segmentId);
+  }
+
+  private static boolean physicalFactAndMeasureMetadataDeletion(String path) {
+
+    boolean status = false;
+    try {
+      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());
+            status = false;
+          }
+        }
+
+      } else {
+        status = false;
+      }
+    } catch (IOException e) {
+      LOGGER.warn("Unable to delete the file as per delete command " + path);
+    }
+
+    return status;
+
+  }
+
+  private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
+      boolean isForceDelete) {
+    if ((CarbonCommonConstants.MARKED_FOR_DELETE.equalsIgnoreCase(oneLoad.getLoadStatus())
+        || CarbonCommonConstants.COMPACTED.equalsIgnoreCase(oneLoad.getLoadStatus()))
+        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      if (isForceDelete) {
+        return true;
+      }
+      long deletionTime = oneLoad.getModificationOrdeletionTimesStamp();
+
+      return CarbonUpdateUtil.isMaxQueryTimeoutExceeded(deletionTime);
+
+    }
+
+    return false;
+  }
+
+  public static boolean deleteLoadFoldersFromFileSystem(String dbName, String tableName,
+      String storeLocation, boolean isForceDelete, LoadMetadataDetails[] details) {
+
+    boolean isDeleted = false;
+
+    if (details != null && details.length != 0) {
+      for (LoadMetadataDetails oneLoad : details) {
+        if (checkIfLoadCanBeDeleted(oneLoad, isForceDelete)) {
+          String path = getSegmentPath(dbName, tableName, storeLocation, 0, oneLoad);
+          boolean deletionStatus = physicalFactAndMeasureMetadataDeletion(path);
+          if (deletionStatus) {
+            isDeleted = true;
+            oneLoad.setVisibility("false");
+            LOGGER.info("Info: Deleted the load " + oneLoad.getLoadName());
+          }
+        }
+      }
+    }
+
+    return isDeleted;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/util/LoadMetadataUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/LoadMetadataUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/LoadMetadataUtil.java
new file mode 100644
index 0000000..415eb8d
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/LoadMetadataUtil.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+
+/**
+ * Utility for load data
+ */
+public final class LoadMetadataUtil {
+  private LoadMetadataUtil() {
+
+  }
+
+  public static boolean isLoadDeletionRequired(String metaDataLocation) {
+    LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(metaDataLocation);
+    if (details != null && details.length != 0) {
+      for (LoadMetadataDetails oneRow : details) {
+        if ((CarbonCommonConstants.MARKED_FOR_DELETE.equalsIgnoreCase(oneRow.getLoadStatus())
+            || CarbonCommonConstants.COMPACTED.equalsIgnoreCase(oneRow.getLoadStatus()))
+            && oneRow.getVisibility().equalsIgnoreCase("true")) {
+          return true;
+        }
+      }
+    }
+
+    return false;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/util/TableOptionConstant.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/TableOptionConstant.java b/processing/src/main/java/org/apache/carbondata/processing/util/TableOptionConstant.java
new file mode 100644
index 0000000..fa910e6
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/TableOptionConstant.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.processing.util;
+
+/**
+ * enum holds the value related to the ddl option
+ */
+public enum TableOptionConstant {
+  SERIALIZATION_NULL_FORMAT("serialization_null_format"),
+  BAD_RECORDS_LOGGER_ENABLE("bad_records_logger_enable"),
+  BAD_RECORDS_ACTION("bad_records_action");
+
+  private String name;
+
+  /**
+   * constructor to initialize the enum value
+   * @param name
+   */
+  TableOptionConstant(String name) {
+    this.name = name;
+  }
+
+  public String getName() {
+    return name;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 3bb186e..61771ea 100644
--- a/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
+++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
@@ -72,16 +72,15 @@ import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWrit
 import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
 import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfo;
 import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortInfoPreparator;
-import org.apache.carbondata.processing.api.dataloader.SchemaInfo;
-import org.apache.carbondata.processing.constants.TableOptionConstant;
-import org.apache.carbondata.processing.csvload.BlockDetails;
-import org.apache.carbondata.processing.csvload.CSVInputFormat;
-import org.apache.carbondata.processing.csvload.CSVRecordReaderIterator;
-import org.apache.carbondata.processing.csvload.StringArrayWritable;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
-import org.apache.carbondata.processing.newflow.DataLoadExecutor;
-import org.apache.carbondata.processing.newflow.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.util.TableOptionConstant;
+import org.apache.carbondata.processing.loading.csvinput.BlockDetails;
+import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat;
+import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator;
+import org.apache.carbondata.processing.loading.csvinput.StringArrayWritable;
+import org.apache.carbondata.processing.loading.model.CarbonDataLoadSchema;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+import org.apache.carbondata.processing.loading.DataLoadExecutor;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
 
 import com.google.gson.Gson;
 import org.apache.hadoop.conf.Configuration;
@@ -384,7 +383,6 @@ public class StoreCreator {
       path.delete();
     }
 
-    SchemaInfo info = new SchemaInfo();
     BlockDetails blockDetails = new BlockDetails(new Path(loadModel.getFactFilePath()),
         0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"});
     Configuration configuration = new Configuration();
@@ -411,9 +409,6 @@ public class StoreCreator {
         storeLocationArray,
         new CarbonIterator[]{readerIterator});
 
-    info.setDatabaseName(databaseName);
-    info.setTableName(tableName);
-
     writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(),
         new ArrayList<LoadMetadataDetails>());
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java b/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java
deleted file mode 100644
index 676838d..0000000
--- a/processing/src/test/java/org/apache/carbondata/processing/csvload/CSVInputFormatTest.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.csvload;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-
-import junit.framework.TestCase;
-import org.junit.Assert;
-import org.junit.Test;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.compress.BZip2Codec;
-import org.apache.hadoop.io.compress.CompressionOutputStream;
-import org.apache.hadoop.io.compress.GzipCodec;
-import org.apache.hadoop.io.compress.Lz4Codec;
-import org.apache.hadoop.io.compress.SnappyCodec;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-
-public class CSVInputFormatTest extends TestCase {
-
-  /**
-   * generate compressed files, no need to call this method.
-   * @throws Exception
-   */
-  public void generateCompressFiles() throws Exception {
-    String pwd = new File("src/test/resources/csv").getCanonicalPath();
-    String inputFile = pwd + "/data.csv";
-    FileInputStream input = new FileInputStream(inputFile);
-    Configuration conf = new Configuration();
-
-    // .gz
-    String outputFile = pwd + "/data.csv.gz";
-    FileOutputStream output = new FileOutputStream(outputFile);
-    GzipCodec gzip = new GzipCodec();
-    gzip.setConf(conf);
-    CompressionOutputStream outputStream = gzip.createOutputStream(output);
-    int i = -1;
-    while ((i = input.read()) != -1) {
-      outputStream.write(i);
-    }
-    outputStream.close();
-    input.close();
-
-    // .bz2
-    input = new FileInputStream(inputFile);
-    outputFile = pwd + "/data.csv.bz2";
-    output = new FileOutputStream(outputFile);
-    BZip2Codec bzip2 = new BZip2Codec();
-    bzip2.setConf(conf);
-    outputStream = bzip2.createOutputStream(output);
-    i = -1;
-    while ((i = input.read()) != -1) {
-      outputStream.write(i);
-    }
-    outputStream.close();
-    input.close();
-
-    // .snappy
-    input = new FileInputStream(inputFile);
-    outputFile = pwd + "/data.csv.snappy";
-    output = new FileOutputStream(outputFile);
-    SnappyCodec snappy = new SnappyCodec();
-    snappy.setConf(conf);
-    outputStream = snappy.createOutputStream(output);
-    i = -1;
-    while ((i = input.read()) != -1) {
-      outputStream.write(i);
-    }
-    outputStream.close();
-    input.close();
-
-    //.lz4
-    input = new FileInputStream(inputFile);
-    outputFile = pwd + "/data.csv.lz4";
-    output = new FileOutputStream(outputFile);
-    Lz4Codec lz4 = new Lz4Codec();
-    lz4.setConf(conf);
-    outputStream = lz4.createOutputStream(output);
-    i = -1;
-    while ((i = input.read()) != -1) {
-      outputStream.write(i);
-    }
-    outputStream.close();
-    input.close();
-
-  }
-
-  /**
-   * CSVCheckMapper check the content of csv files.
-   */
-  public static class CSVCheckMapper extends Mapper<NullWritable, StringArrayWritable, NullWritable,
-      NullWritable> {
-    @Override
-    protected void map(NullWritable key, StringArrayWritable value, Context context)
-        throws IOException, InterruptedException {
-      String[] columns = value.get();
-      int id = Integer.parseInt(columns[0]);
-      int salary = Integer.parseInt(columns[6]);
-      Assert.assertEquals(id - 1, salary - 15000);
-    }
-  }
-
-  /**
-   * test read csv files
-   * @throws Exception
-   */
-  @Test public void testReadCSVFiles() throws Exception{
-    Configuration conf = new Configuration();
-    prepareConf(conf);
-    Job job = Job.getInstance(conf, "CSVInputFormat_normal");
-    job.setJarByClass(CSVInputFormatTest.class);
-    job.setMapperClass(CSVCheckMapper.class);
-    job.setNumReduceTasks(0);
-    job.setInputFormatClass(CSVInputFormat.class);
-
-    String inputFolder = new File("src/test/resources/csv").getCanonicalPath();
-    FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv"));
-    FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv.bz2"));
-    FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv.gz"));
-    // FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv.lz4"));
-    // FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv.snappy"));
-
-    File output = new File("target/output_CSVInputFormatTest");
-    deleteOutput(output);
-    FileOutputFormat.setOutputPath(job, new Path(output.getCanonicalPath()));
-
-    Assert.assertTrue(job.waitForCompletion(true));
-  }
-
-  private void prepareConf(Configuration conf) {
-    conf.setBoolean(CSVInputFormat.HEADER_PRESENT, true);
-    conf.set(CSVInputFormat.MAX_COLUMNS, "10");
-    conf.set(CSVInputFormat.NUMBER_OF_COLUMNS, "7");
-  }
-
-  private void deleteOutput(File output) {
-    if (output.exists()) {
-      if (output.isDirectory()) {
-        for(File file : output.listFiles()) {
-          deleteOutput(file);
-        }
-        output.delete();
-      } else {
-        output.delete();
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..925701d
--- /dev/null
+++ b/processing/src/test/java/org/apache/carbondata/processing/loading/csvinput/CSVInputFormatTest.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.csvinput;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import junit.framework.TestCase;
+import org.junit.Assert;
+import org.junit.Test;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.compress.BZip2Codec;
+import org.apache.hadoop.io.compress.CompressionOutputStream;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.compress.Lz4Codec;
+import org.apache.hadoop.io.compress.SnappyCodec;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+public class CSVInputFormatTest extends TestCase {
+
+  /**
+   * generate compressed files, no need to call this method.
+   * @throws Exception
+   */
+  public void generateCompressFiles() throws Exception {
+    String pwd = new File("src/test/resources/csv").getCanonicalPath();
+    String inputFile = pwd + "/data.csv";
+    FileInputStream input = new FileInputStream(inputFile);
+    Configuration conf = new Configuration();
+
+    // .gz
+    String outputFile = pwd + "/data.csv.gz";
+    FileOutputStream output = new FileOutputStream(outputFile);
+    GzipCodec gzip = new GzipCodec();
+    gzip.setConf(conf);
+    CompressionOutputStream outputStream = gzip.createOutputStream(output);
+    int i = -1;
+    while ((i = input.read()) != -1) {
+      outputStream.write(i);
+    }
+    outputStream.close();
+    input.close();
+
+    // .bz2
+    input = new FileInputStream(inputFile);
+    outputFile = pwd + "/data.csv.bz2";
+    output = new FileOutputStream(outputFile);
+    BZip2Codec bzip2 = new BZip2Codec();
+    bzip2.setConf(conf);
+    outputStream = bzip2.createOutputStream(output);
+    i = -1;
+    while ((i = input.read()) != -1) {
+      outputStream.write(i);
+    }
+    outputStream.close();
+    input.close();
+
+    // .snappy
+    input = new FileInputStream(inputFile);
+    outputFile = pwd + "/data.csv.snappy";
+    output = new FileOutputStream(outputFile);
+    SnappyCodec snappy = new SnappyCodec();
+    snappy.setConf(conf);
+    outputStream = snappy.createOutputStream(output);
+    i = -1;
+    while ((i = input.read()) != -1) {
+      outputStream.write(i);
+    }
+    outputStream.close();
+    input.close();
+
+    //.lz4
+    input = new FileInputStream(inputFile);
+    outputFile = pwd + "/data.csv.lz4";
+    output = new FileOutputStream(outputFile);
+    Lz4Codec lz4 = new Lz4Codec();
+    lz4.setConf(conf);
+    outputStream = lz4.createOutputStream(output);
+    i = -1;
+    while ((i = input.read()) != -1) {
+      outputStream.write(i);
+    }
+    outputStream.close();
+    input.close();
+
+  }
+
+  /**
+   * CSVCheckMapper check the content of csv files.
+   */
+  public static class CSVCheckMapper extends Mapper<NullWritable, StringArrayWritable, NullWritable,
+      NullWritable> {
+    @Override
+    protected void map(NullWritable key, StringArrayWritable value, Context context)
+        throws IOException, InterruptedException {
+      String[] columns = value.get();
+      int id = Integer.parseInt(columns[0]);
+      int salary = Integer.parseInt(columns[6]);
+      Assert.assertEquals(id - 1, salary - 15000);
+    }
+  }
+
+  /**
+   * test read csv files
+   * @throws Exception
+   */
+  @Test public void testReadCSVFiles() throws Exception{
+    Configuration conf = new Configuration();
+    prepareConf(conf);
+    Job job = Job.getInstance(conf, "CSVInputFormat_normal");
+    job.setJarByClass(CSVInputFormatTest.class);
+    job.setMapperClass(CSVCheckMapper.class);
+    job.setNumReduceTasks(0);
+    job.setInputFormatClass(CSVInputFormat.class);
+
+    String inputFolder = new File("src/test/resources/csv").getCanonicalPath();
+    FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv"));
+    FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv.bz2"));
+    FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv.gz"));
+    // FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv.lz4"));
+    // FileInputFormat.addInputPath(job, new Path(inputFolder + File.separator + "data.csv.snappy"));
+
+    File output = new File("target/output_CSVInputFormatTest");
+    deleteOutput(output);
+    FileOutputFormat.setOutputPath(job, new Path(output.getCanonicalPath()));
+
+    Assert.assertTrue(job.waitForCompletion(true));
+  }
+
+  private void prepareConf(Configuration conf) {
+    conf.setBoolean(CSVInputFormat.HEADER_PRESENT, true);
+    conf.set(CSVInputFormat.MAX_COLUMNS, "10");
+    conf.set(CSVInputFormat.NUMBER_OF_COLUMNS, "7");
+  }
+
+  private void deleteOutput(File output) {
+    if (output.exists()) {
+      if (output.isDirectory()) {
+        for(File file : output.listFiles()) {
+          deleteOutput(file);
+        }
+        output.delete();
+      } else {
+        output.delete();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/test/java/org/apache/carbondata/processing/loading/dictionary/InMemBiDictionaryTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/loading/dictionary/InMemBiDictionaryTest.java b/processing/src/test/java/org/apache/carbondata/processing/loading/dictionary/InMemBiDictionaryTest.java
new file mode 100644
index 0000000..88ff377
--- /dev/null
+++ b/processing/src/test/java/org/apache/carbondata/processing/loading/dictionary/InMemBiDictionaryTest.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.processing.loading.dictionary;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.carbondata.core.devapi.BiDictionary;
+import org.apache.carbondata.core.devapi.DictionaryGenerationException;
+import org.apache.carbondata.core.devapi.DictionaryGenerator;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class InMemBiDictionaryTest {
+
+  /**
+   * test pre-created dictionary
+   */
+  @Test public void testPreCreated() throws Exception {
+    Map<Integer, String> map = new HashMap<>();
+    map.put(1, "amy");
+    map.put(2, "bob");
+    BiDictionary<Integer, String> dict = new InMemBiDictionary<>(map);
+    Assert.assertEquals(1, dict.getKey("amy").intValue());
+    Assert.assertEquals(2, dict.getKey("bob").intValue());
+    Assert.assertEquals("amy", dict.getValue(1));
+    Assert.assertEquals("bob", dict.getValue(2));
+    Assert.assertEquals(2, dict.size());
+    try {
+      dict.getOrGenerateKey("cat");
+      Assert.fail("add dictionary successfully");
+    } catch (Exception e) {
+      // test pass
+    }
+  }
+
+  /**
+   * test generating dictionary on the fly
+   */
+  @Test public void testGenerateDict() throws Exception {
+    BiDictionary<Integer, String> dict = new InMemBiDictionary<>(
+        new DictionaryGenerator<Integer, String>() {
+          int sequence = 1;
+          @Override
+          public Integer generateKey(String value) throws DictionaryGenerationException {
+            return sequence++;
+          }
+        });
+    Assert.assertEquals(1, dict.getOrGenerateKey("amy").intValue());
+    Assert.assertEquals(2, dict.getOrGenerateKey("bob").intValue());
+    Assert.assertEquals(1, dict.getKey("amy").intValue());
+    Assert.assertEquals(2, dict.getKey("bob").intValue());
+    Assert.assertEquals("amy", dict.getValue(1));
+    Assert.assertEquals("bob", dict.getValue(2));
+    Assert.assertEquals(2, dict.size());
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/test/java/org/apache/carbondata/processing/newflow/dictionary/InMemBiDictionaryTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/processing/newflow/dictionary/InMemBiDictionaryTest.java b/processing/src/test/java/org/apache/carbondata/processing/newflow/dictionary/InMemBiDictionaryTest.java
deleted file mode 100644
index 6d82cce..0000000
--- a/processing/src/test/java/org/apache/carbondata/processing/newflow/dictionary/InMemBiDictionaryTest.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow.dictionary;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.carbondata.core.devapi.BiDictionary;
-import org.apache.carbondata.core.devapi.DictionaryGenerationException;
-import org.apache.carbondata.core.devapi.DictionaryGenerator;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class InMemBiDictionaryTest {
-
-  /**
-   * test pre-created dictionary
-   */
-  @Test public void testPreCreated() throws Exception {
-    Map<Integer, String> map = new HashMap<>();
-    map.put(1, "amy");
-    map.put(2, "bob");
-    BiDictionary<Integer, String> dict = new InMemBiDictionary<>(map);
-    Assert.assertEquals(1, dict.getKey("amy").intValue());
-    Assert.assertEquals(2, dict.getKey("bob").intValue());
-    Assert.assertEquals("amy", dict.getValue(1));
-    Assert.assertEquals("bob", dict.getValue(2));
-    Assert.assertEquals(2, dict.size());
-    try {
-      dict.getOrGenerateKey("cat");
-      Assert.fail("add dictionary successfully");
-    } catch (Exception e) {
-      // test pass
-    }
-  }
-
-  /**
-   * test generating dictionary on the fly
-   */
-  @Test public void testGenerateDict() throws Exception {
-    BiDictionary<Integer, String> dict = new InMemBiDictionary<>(
-        new DictionaryGenerator<Integer, String>() {
-          int sequence = 1;
-          @Override
-          public Integer generateKey(String value) throws DictionaryGenerationException {
-            return sequence++;
-          }
-        });
-    Assert.assertEquals(1, dict.getOrGenerateKey("amy").intValue());
-    Assert.assertEquals(2, dict.getOrGenerateKey("bob").intValue());
-    Assert.assertEquals(1, dict.getKey("amy").intValue());
-    Assert.assertEquals(2, dict.getKey("bob").intValue());
-    Assert.assertEquals("amy", dict.getValue(1));
-    Assert.assertEquals("bob", dict.getValue(2));
-    Assert.assertEquals(2, dict.size());
-  }
-}


[43/50] [abbrv] carbondata git commit: [CARBONDATA-1410] Fixed thread leak issue in case of data loading

Posted by ja...@apache.org.
[CARBONDATA-1410] Fixed thread leak issue in case of data loading

Problem: In case of data loading failure threads are not getting closed and its causing thread leak in long run, because of this OOM is coming
Solution: Close all the thread in case of failure , Success and killing

This closes #1401


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

Branch: refs/heads/streaming_ingest
Commit: a734add5a95790f207d21a2e0dcc4e1480d51932
Parents: 7d3bf05
Author: kumarvishal <ku...@gmail.com>
Authored: Mon Sep 25 18:24:03 2017 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Oct 5 16:38:48 2017 +0530

----------------------------------------------------------------------
 .../exception/CarbonDataWriterException.java    |  9 +++
 .../core/util/CarbonThreadFactory.java          | 47 ++++++++++++
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |  9 ++-
 .../converter/impl/RowConverterImpl.java        |  5 +-
 .../sort/impl/ParallelReadMergeSorterImpl.java  |  9 ++-
 .../impl/UnsafeParallelReadMergeSorterImpl.java |  9 ++-
 .../loading/sort/unsafe/UnsafeSortDataRows.java | 11 ++-
 .../holder/UnsafeSortTempFileChunkHolder.java   |  4 +-
 .../merger/UnsafeIntermediateFileMerger.java    | 40 ++++++----
 .../unsafe/merger/UnsafeIntermediateMerger.java | 31 +++++---
 .../UnsafeSingleThreadFinalSortFilesMerger.java |  3 +-
 .../loading/steps/InputProcessorStepImpl.java   |  5 +-
 .../sort/sortdata/IntermediateFileMerger.java   | 43 +++++++----
 .../SingleThreadFinalSortFilesMerger.java       | 81 ++++++++++++++------
 .../processing/sort/sortdata/SortDataRows.java  | 23 ++++--
 .../sortdata/SortIntermediateFileMerger.java    | 25 +++++-
 .../sort/sortdata/SortTempFileChunkHolder.java  | 11 ++-
 .../store/CarbonFactDataHandlerColumnar.java    | 19 ++++-
 .../store/writer/AbstractFactDataWriter.java    |  4 +-
 .../writer/v3/CarbonFactDataWriterImplV3.java   |  6 +-
 .../processing/util/CarbonLoaderUtil.java       |  4 +-
 21 files changed, 308 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/core/src/main/java/org/apache/carbondata/core/datastore/exception/CarbonDataWriterException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/exception/CarbonDataWriterException.java b/core/src/main/java/org/apache/carbondata/core/datastore/exception/CarbonDataWriterException.java
index 8f75ee8..f9f44a4 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/exception/CarbonDataWriterException.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/exception/CarbonDataWriterException.java
@@ -44,6 +44,15 @@ public class CarbonDataWriterException extends RuntimeException {
   /**
    * Constructor
    *
+   * @param t exception.
+   */
+  public CarbonDataWriterException(Throwable t) {
+    super(t);
+  }
+
+  /**
+   * Constructor
+   *
    * @param msg The error message for this exception.
    */
   public CarbonDataWriterException(String msg, Throwable t) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java
new file mode 100644
index 0000000..689365d
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonThreadFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+
+/**
+ * Carbon thread factory class
+ */
+public class CarbonThreadFactory implements ThreadFactory {
+
+  /**
+   * default thread factory
+   */
+  private ThreadFactory defaultFactory;
+
+  /**
+   * pool name
+   */
+  private String name;
+
+  public CarbonThreadFactory(String name) {
+    this.defaultFactory = Executors.defaultThreadFactory();
+    this.name = name;
+  }
+
+  @Override public Thread newThread(Runnable r) {
+    final Thread thread = defaultFactory.newThread(r);
+    thread.setName(name);
+    return thread;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 1d1b47a..49b708c 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
@@ -299,8 +299,9 @@ class NewCarbonDataLoadRDD[K, V](
                 split.serializableHadoopSplit.value.getPartition.getUniqueID)
           }
           partitionID = split.serializableHadoopSplit.value.getPartition.getUniqueID
-
-          StandardLogService.setThreadName(partitionID, null)
+          StandardLogService.setThreadName(StandardLogService
+            .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)
+            , ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId + "")
           CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordPartitionBlockMap(
               partitionID, split.partitionBlocksDetail.length)
           val readers =
@@ -328,7 +329,9 @@ class NewCarbonDataLoadRDD[K, V](
           } else {
             model = carbonLoadModel.getCopyWithPartition(partitionID)
           }
-          StandardLogService.setThreadName(blocksID, null)
+          StandardLogService.setThreadName(StandardLogService
+            .getPartitionID(model.getCarbonDataLoadSchema.getCarbonTable.getTableUniqueName)
+            , ThreadLocalTaskInfo.getCarbonTaskInfo.getTaskId + "")
           val readers =
             split.nodeBlocksDetail.map(format.createRecordReader(_, hadoopAttemptContext))
           readers.zipWithIndex.map { case (reader, index) =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 a4351ae..79c6d61 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
@@ -36,6 +36,7 @@ import org.apache.carbondata.core.cache.dictionary.Dictionary;
 import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.dictionary.client.DictionaryClient;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.processing.loading.BadRecordsLogger;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
@@ -117,7 +118,9 @@ public class RowConverterImpl implements RowConverter {
     // for one pass load, start the dictionary client
     if (configuration.getUseOnePass()) {
       if (executorService == null) {
-        executorService = Executors.newCachedThreadPool();
+        executorService = Executors.newCachedThreadPool(new CarbonThreadFactory(
+            "DictionaryClientPool:" + configuration.getTableIdentifier().getCarbonTableIdentifier()
+                .getTableName()));
       }
       Future<DictionaryClient> result = executorService.submit(new Callable<DictionaryClient>() {
         @Override

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 6e43fcb..cefc97d 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
@@ -30,6 +30,7 @@ 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.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
@@ -59,6 +60,8 @@ public class ParallelReadMergeSorterImpl extends AbstractMergeSorter {
 
   private AtomicLong rowCounter;
 
+  private ExecutorService executorService;
+
   public ParallelReadMergeSorterImpl(AtomicLong rowCounter) {
     this.rowCounter = rowCounter;
   }
@@ -94,7 +97,8 @@ public class ParallelReadMergeSorterImpl extends AbstractMergeSorter {
     } catch (CarbonSortKeyAndGroupByException e) {
       throw new CarbonDataLoadingException(e);
     }
-    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
+    this.executorService = Executors.newFixedThreadPool(iterators.length,
+        new CarbonThreadFactory("SafeParallelSorterPool:" + sortParameters.getTableName()));
     this.threadStatusObserver = new ThreadStatusObserver(executorService);
 
     try {
@@ -147,6 +151,9 @@ public class ParallelReadMergeSorterImpl extends AbstractMergeSorter {
     if (intermediateFileMerger != null) {
       intermediateFileMerger.close();
     }
+    if (null != executorService && !executorService.isShutdown()) {
+      executorService.shutdownNow();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java
index 1a2f704..c05c027 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/sort/impl/UnsafeParallelReadMergeSorterImpl.java
@@ -30,6 +30,7 @@ import org.apache.carbondata.core.datastore.exception.CarbonDataWriterException;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
 import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
 import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
@@ -59,6 +60,8 @@ public class UnsafeParallelReadMergeSorterImpl extends AbstractMergeSorter {
 
   private AtomicLong rowCounter;
 
+  private ExecutorService executorService;
+
   public UnsafeParallelReadMergeSorterImpl(AtomicLong rowCounter) {
     this.rowCounter = rowCounter;
   }
@@ -82,7 +85,8 @@ public class UnsafeParallelReadMergeSorterImpl extends AbstractMergeSorter {
     } catch (MemoryException e) {
       throw new CarbonDataLoadingException(e);
     }
-    ExecutorService executorService = Executors.newFixedThreadPool(iterators.length);
+    this.executorService = Executors.newFixedThreadPool(iterators.length,
+        new CarbonThreadFactory("UnsafeParallelSorterPool:" + sortParameters.getTableName()));
     this.threadStatusObserver = new ThreadStatusObserver(executorService);
 
     try {
@@ -131,6 +135,9 @@ public class UnsafeParallelReadMergeSorterImpl extends AbstractMergeSorter {
   }
 
   @Override public void close() {
+    if (null != executorService && !executorService.isShutdown()) {
+      executorService.shutdownNow();
+    }
     unsafeIntermediateFileMerger.close();
     finalMerger.clear();
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 88b72aa..0210464 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
@@ -38,6 +38,7 @@ import org.apache.carbondata.core.memory.MemoryException;
 import org.apache.carbondata.core.memory.UnsafeMemoryManager;
 import org.apache.carbondata.core.memory.UnsafeSortMemoryManager;
 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.ThreadLocalTaskInfo;
 import org.apache.carbondata.processing.loading.sort.unsafe.comparator.UnsafeRowComparator;
@@ -137,8 +138,9 @@ public class UnsafeSortDataRows {
 
     // create new sort temp directory
     CarbonDataProcessorUtil.createLocations(parameters.getTempFileLocation());
-    this.dataSorterAndWriterExecutorService =
-        Executors.newFixedThreadPool(parameters.getNumberOfCores());
+    this.dataSorterAndWriterExecutorService = Executors
+        .newFixedThreadPool(parameters.getNumberOfCores(),
+            new CarbonThreadFactory("UnsafeSortDataRowPool:" + parameters.getTableName()));
     semaphore = new Semaphore(parameters.getNumberOfCores());
   }
 
@@ -372,7 +374,8 @@ public class UnsafeSortDataRows {
                   + System.nanoTime() + CarbonCommonConstants.SORT_TEMP_FILE_EXT);
           writeData(page, sortTempFile);
           LOGGER.info("Time taken to sort row page with size" + page.getBuffer().getActualSize()
-              + " and write is: " + (System.currentTimeMillis() - startTime));
+              + " and write is: " + (System.currentTimeMillis() - startTime) + ": location:"
+              + 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
@@ -395,7 +398,7 @@ public class UnsafeSortDataRows {
           page.getBuffer().loadToUnsafe();
           unsafeInMemoryIntermediateFileMerger.addDataChunkToMerge(page);
           LOGGER.info(
-              "Time taken to sort row page with size" + page.getBuffer().getActualSize() + "is: "
+              "Time taken to sort row page with size: " + page.getBuffer().getActualSize() + "is: "
                   + (System.currentTimeMillis() - startTime));
         }
       } catch (Throwable e) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 331b9db..5fed2ea 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
@@ -383,7 +383,9 @@ public class UnsafeSortTempFileChunkHolder implements SortTempChunkHolder {
    */
   public void close() {
     CarbonUtil.closeStreams(stream);
-    executorService.shutdown();
+    if (null != executorService && !executorService.isShutdown()) {
+      executorService.shutdownNow();
+    }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 7f98d72..4303ec8 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
@@ -27,6 +27,7 @@ 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;
@@ -40,7 +41,7 @@ import org.apache.carbondata.processing.sort.sortdata.SortParameters;
 import org.apache.carbondata.processing.sort.sortdata.TempSortFileWriter;
 import org.apache.carbondata.processing.sort.sortdata.TempSortFileWriterFactory;
 
-public class UnsafeIntermediateFileMerger implements Runnable {
+public class UnsafeIntermediateFileMerger implements Callable<Void> {
   /**
    * LOGGER
    */
@@ -84,6 +85,8 @@ public class UnsafeIntermediateFileMerger implements Runnable {
 
   private ByteBuffer rowData;
 
+  private Throwable throwable;
+
   /**
    * IntermediateFileMerger Constructor
    */
@@ -99,11 +102,9 @@ public class UnsafeIntermediateFileMerger implements Runnable {
     rowData = ByteBuffer.allocate(2 * 1024 * 1024);
   }
 
-  @Override
-  public void run() {
+  @Override public Void call() throws Exception {
     long intermediateMergeStartTime = System.currentTimeMillis();
     int fileConterConst = fileCounter;
-    boolean isFailed = false;
     try {
       startSorting();
       initialize();
@@ -116,24 +117,30 @@ public class UnsafeIntermediateFileMerger implements Runnable {
           + " Sort Temp Files Cost Time: " + intermediateMergeCostTime + "(s)");
     } catch (Exception e) {
       LOGGER.error(e, "Problem while intermediate merging");
-      isFailed = true;
+      clear();
+      throwable = e;
     } finally {
       CarbonUtil.closeStreams(this.stream);
       if (null != writer) {
         writer.finish();
       }
-      if (!isFailed) {
+      if (null == throwable) {
         try {
           finish();
         } catch (CarbonSortKeyAndGroupByException e) {
           LOGGER.error(e, "Problem while deleting the merge file");
+          throwable = e;
         }
       } else {
-        if (outPutFile.delete()) {
+        if (!outPutFile.delete()) {
           LOGGER.error("Problem while deleting the merge file");
         }
       }
     }
+    if (null != throwable) {
+      throw new CarbonSortKeyAndGroupByException(throwable);
+    }
+    return null;
   }
 
   /**
@@ -351,12 +358,7 @@ public class UnsafeIntermediateFileMerger implements Runnable {
   }
 
   private void finish() throws CarbonSortKeyAndGroupByException {
-    if (recordHolderHeap != null) {
-      int size = recordHolderHeap.size();
-      for (int i = 0; i < size; i++) {
-        recordHolderHeap.poll().close();
-      }
-    }
+    clear();
     try {
       CarbonUtil.deleteFiles(intermediateFiles);
       rowData.clear();
@@ -364,4 +366,16 @@ public class UnsafeIntermediateFileMerger implements Runnable {
       throw new CarbonSortKeyAndGroupByException("Problem while deleting the intermediate files");
     }
   }
+
+  private void clear() {
+    if (null != recordHolderHeap) {
+      SortTempChunkHolder sortTempChunkHolder;
+      while (!recordHolderHeap.isEmpty()) {
+        sortTempChunkHolder = recordHolderHeap.poll();
+        if (null != sortTempChunkHolder) {
+          sortTempChunkHolder.close();
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 c774d8f..0d24e01 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
@@ -20,14 +20,16 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
+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;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.processing.loading.sort.unsafe.UnsafeCarbonRowPage;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 import org.apache.carbondata.processing.sort.sortdata.SortParameters;
@@ -55,20 +57,19 @@ public class UnsafeIntermediateMerger {
 
   private final Object lockObject = new Object();
 
-  private boolean offHeap;
-
   private List<File> procFiles;
 
+  private List<Future<Void>> mergerTask;
+
   public UnsafeIntermediateMerger(SortParameters parameters) {
     this.parameters = parameters;
     // processed file list
     this.rowPages = new ArrayList<UnsafeCarbonRowPage>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
     this.mergedPages = new ArrayList<>();
-    this.executorService = Executors.newFixedThreadPool(parameters.getNumberOfCores());
-    this.offHeap = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.ENABLE_OFFHEAP_SORT,
-            CarbonCommonConstants.ENABLE_OFFHEAP_SORT_DEFAULT));
+    this.executorService = Executors.newFixedThreadPool(parameters.getNumberOfCores(),
+        new CarbonThreadFactory("UnsafeIntermediatePool:" + parameters.getTableName()));
     this.procFiles = new ArrayList<File>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    this.mergerTask = new ArrayList<>();
   }
 
   public void addDataChunkToMerge(UnsafeCarbonRowPage rowPage) {
@@ -116,7 +117,7 @@ public class UnsafeIntermediateMerger {
             .nanoTime() + CarbonCommonConstants.MERGERD_EXTENSION);
     UnsafeIntermediateFileMerger merger =
         new UnsafeIntermediateFileMerger(parameters, intermediateFiles, file);
-    executorService.execute(merger);
+    mergerTask.add(executorService.submit(merger));
   }
 
   public void startInmemoryMergingIfPossible() throws CarbonSortKeyAndGroupByException {
@@ -167,16 +168,28 @@ public class UnsafeIntermediateMerger {
     } catch (InterruptedException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while shutdown the server ", e);
     }
+    checkForFailure();
   }
 
   public void close() {
-    if (executorService.isShutdown()) {
+    if (!executorService.isShutdown()) {
       executorService.shutdownNow();
     }
     rowPages.clear();
     rowPages = null;
   }
 
+  private void checkForFailure() throws CarbonSortKeyAndGroupByException {
+    for (int i = 0; i < mergerTask.size(); i++) {
+      try {
+        mergerTask.get(i).get();
+      } catch (InterruptedException | ExecutionException e) {
+        LOGGER.error(e, e.getMessage());
+        throw new CarbonSortKeyAndGroupByException(e.getMessage(), e);
+      }
+    }
+  }
+
   public List<UnsafeCarbonRowPage> getRowPages() {
     return rowPages;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 32b31d7..eb38efe 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
@@ -97,6 +97,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
         LOGGER.info("No files to merge sort");
         return;
       }
+      LOGGER.info("Starting final merger");
       LOGGER.info("Number of row pages: " + this.fileCounter);
 
       // create record holder heap
@@ -143,7 +144,7 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
     } catch (Exception e) {
       LOGGER.error(e);
-      throw new CarbonDataWriterException(e.getMessage());
+      throw new CarbonDataWriterException(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java
index 70a1254..ae7ece1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.core.datastore.row.CarbonRow;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
 import org.apache.carbondata.processing.loading.DataField;
@@ -64,7 +65,9 @@ public class InputProcessorStepImpl extends AbstractDataLoadProcessorStep {
   @Override public void initialize() throws IOException {
     super.initialize();
     rowParser = new RowParserImpl(getOutput(), configuration);
-    executorService = Executors.newCachedThreadPool();
+    executorService = Executors.newCachedThreadPool(new CarbonThreadFactory(
+        "InputProcessorPool:" + configuration.getTableIdentifier().getCarbonTableIdentifier()
+            .getTableName()));
   }
 
   @Override public Iterator<CarbonRowBatch>[] execute() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 ffe6fb6..d4a8dd6 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
@@ -25,6 +25,7 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.util.AbstractQueue;
 import java.util.PriorityQueue;
+import java.util.concurrent.Callable;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -33,7 +34,7 @@ import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.NonDictionaryUtil;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
-public class IntermediateFileMerger implements Runnable {
+public class IntermediateFileMerger implements Callable<Void> {
   /**
    * LOGGER
    */
@@ -88,6 +89,8 @@ public class IntermediateFileMerger implements Runnable {
 
   private boolean[] noDictionarycolumnMapping;
 
+  private Throwable throwable;
+
   /**
    * IntermediateFileMerger Constructor
    */
@@ -100,11 +103,9 @@ public class IntermediateFileMerger implements Runnable {
     noDictionarycolumnMapping = mergerParameters.getNoDictionaryDimnesionColumn();
   }
 
-  @Override
-  public void run() {
+  @Override public Void call() throws Exception {
     long intermediateMergeStartTime = System.currentTimeMillis();
     int fileConterConst = fileCounter;
-    boolean isFailed = false;
     try {
       startSorting();
       initialize();
@@ -129,25 +130,31 @@ public class IntermediateFileMerger implements Runnable {
           " Sort Temp Files Cost Time: " + intermediateMergeCostTime + "(s)");
     } catch (Exception e) {
       LOGGER.error(e, "Problem while intermediate merging");
-      isFailed = true;
+      clear();
+      throwable = e;
     } finally {
       records = null;
       CarbonUtil.closeStreams(this.stream);
       if (null != writer) {
         writer.finish();
       }
-      if (!isFailed) {
+      if (null == throwable) {
         try {
           finish();
         } catch (CarbonSortKeyAndGroupByException e) {
           LOGGER.error(e, "Problem while deleting the merge file");
+          throwable = e;
         }
       } else {
-        if (outPutFile.delete()) {
+        if (!outPutFile.delete()) {
           LOGGER.error("Problem while deleting the merge file");
         }
       }
     }
+    if (null != throwable) {
+      throw new CarbonSortKeyAndGroupByException(throwable);
+    }
+    return null;
   }
 
   /**
@@ -251,7 +258,7 @@ public class IntermediateFileMerger implements Runnable {
               mergerParameters.getFileBufferSize(), mergerParameters.getNoDictionaryCount(),
               mergerParameters.getMeasureDataType(),
               mergerParameters.getNoDictionaryDimnesionColumn(),
-              mergerParameters.getNoDictionarySortColumn());
+              mergerParameters.getNoDictionarySortColumn(), mergerParameters.getTableName());
 
       // initialize
       sortTempFileChunkHolder.initialize();
@@ -370,16 +377,24 @@ public class IntermediateFileMerger implements Runnable {
   }
 
   private void finish() throws CarbonSortKeyAndGroupByException {
-    if (recordHolderHeap != null) {
-      int size = recordHolderHeap.size();
-      for (int i = 0; i < size; i++) {
-        recordHolderHeap.poll().closeStream();
-      }
-    }
+    clear();
     try {
       CarbonUtil.deleteFiles(intermediateFiles);
     } catch (IOException e) {
       throw new CarbonSortKeyAndGroupByException("Problem while deleting the intermediate files");
     }
   }
+
+  private void clear() {
+    if (recordHolderHeap != null) {
+      SortTempFileChunkHolder sortTempFileChunkHolder;
+      while (!recordHolderHeap.isEmpty()) {
+        sortTempFileChunkHolder = recordHolderHeap.poll();
+        if (null != sortTempFileChunkHolder) {
+          sortTempFileChunkHolder.closeStream();
+        }
+      }
+    }
+    recordHolderHeap = null;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 6d6ff94..db4c771 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
@@ -24,8 +24,11 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.PriorityQueue;
+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.CarbonIterator;
@@ -105,6 +108,12 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
 
   private boolean[] isNoDictionarySortColumn;
 
+  private int maxThreadForSorting;
+
+  private ExecutorService executorService;
+
+  private List<Future<Void>> mergerTask;
+
   public SingleThreadFinalSortFilesMerger(String[] tempFileLocation, String tableName,
       int dimensionCount, int complexDimensionCount, int measureCount, int noDictionaryCount,
       DataType[] type, boolean[] isNoDictionaryColumn, boolean[] isNoDictionarySortColumn) {
@@ -117,6 +126,15 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
     this.noDictionaryCount = noDictionaryCount;
     this.isNoDictionaryColumn = isNoDictionaryColumn;
     this.isNoDictionarySortColumn = isNoDictionarySortColumn;
+    try {
+      maxThreadForSorting = Integer.parseInt(CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD,
+              CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE));
+    } catch (NumberFormatException e) {
+      maxThreadForSorting =
+          Integer.parseInt(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE);
+    }
+    this.mergerTask = new ArrayList<>();
   }
 
   /**
@@ -174,6 +192,8 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
         .getFileBufferSize(this.fileCounter, CarbonProperties.getInstance(),
             CarbonCommonConstants.CONSTANT_SIZE_TEN);
 
+    LOGGER.info("Started Final Merge");
+
     LOGGER.info("Number of temp file: " + this.fileCounter);
 
     LOGGER.info("File Buffer Size: " + this.fileBufferSize);
@@ -183,51 +203,51 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
 
     // iterate over file list and create chunk holder and add to heap
     LOGGER.info("Started adding first record from each file");
-    int maxThreadForSorting = 0;
-    try {
-      maxThreadForSorting = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD,
-              CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE));
-    } catch (NumberFormatException e) {
-      maxThreadForSorting =
-          Integer.parseInt(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE);
-    }
-    ExecutorService service = Executors.newFixedThreadPool(maxThreadForSorting);
+    this.executorService = Executors.newFixedThreadPool(maxThreadForSorting);
 
     for (final File tempFile : files) {
 
-      Runnable runnable = new Runnable() {
-        @Override public void run() {
-
+      Callable<Void> callable = new Callable<Void>() {
+        @Override public Void call() throws CarbonSortKeyAndGroupByException {
             // create chunk holder
             SortTempFileChunkHolder sortTempFileChunkHolder =
                 new SortTempFileChunkHolder(tempFile, dimensionCount, complexDimensionCount,
                     measureCount, fileBufferSize, noDictionaryCount, measureDataType,
-                    isNoDictionaryColumn, isNoDictionarySortColumn);
+                    isNoDictionaryColumn, isNoDictionarySortColumn, tableName);
           try {
             // initialize
             sortTempFileChunkHolder.initialize();
             sortTempFileChunkHolder.readRow();
           } catch (CarbonSortKeyAndGroupByException ex) {
-            LOGGER.error(ex);
+            sortTempFileChunkHolder.closeStream();
+            notifyFailure(ex);
           }
-
           synchronized (LOCKOBJECT) {
             recordHolderHeapLocal.add(sortTempFileChunkHolder);
           }
+          return null;
         }
       };
-      service.execute(runnable);
+      mergerTask.add(executorService.submit(callable));
     }
-    service.shutdown();
-
+    executorService.shutdown();
     try {
-      service.awaitTermination(2, TimeUnit.HOURS);
+      executorService.awaitTermination(2, TimeUnit.HOURS);
     } catch (Exception e) {
       throw new CarbonDataWriterException(e.getMessage(), e);
     }
+    checkFailure();
+    LOGGER.info("final merger Heap Size" + this.recordHolderHeapLocal.size());
+  }
 
-    LOGGER.info("Heap Size" + this.recordHolderHeapLocal.size());
+  private void checkFailure() {
+    for (int i = 0; i < mergerTask.size(); i++) {
+      try {
+        mergerTask.get(i).get();
+      } catch (InterruptedException | ExecutionException e) {
+        throw new CarbonDataWriterException(e);
+      }
+    }
   }
 
   /**
@@ -239,6 +259,11 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
     this.recordHolderHeapLocal = new PriorityQueue<SortTempFileChunkHolder>(fileCounter);
   }
 
+  private synchronized void notifyFailure(Throwable throwable) {
+    close();
+    LOGGER.error(throwable);
+  }
+
   /**
    * This method will be used to get the sorted row
    *
@@ -284,6 +309,7 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
     try {
       poll.readRow();
     } catch (CarbonSortKeyAndGroupByException e) {
+      close();
       throw new CarbonDataWriterException(e.getMessage(), e);
     }
 
@@ -304,9 +330,18 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
     return this.fileCounter > 0;
   }
 
-  public void clear() {
+  public void close() {
+    if (null != executorService && !executorService.isShutdown()) {
+      executorService.shutdownNow();
+    }
     if (null != recordHolderHeapLocal) {
-      recordHolderHeapLocal = null;
+      SortTempFileChunkHolder sortTempFileChunkHolder;
+      while (!recordHolderHeapLocal.isEmpty()) {
+        sortTempFileChunkHolder = recordHolderHeapLocal.poll();
+        if (null != sortTempFileChunkHolder) {
+          sortTempFileChunkHolder.closeStream();
+        }
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 fc744a6..11df276 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
@@ -35,6 +35,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 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.sort.exception.CarbonSortKeyAndGroupByException;
@@ -102,8 +103,9 @@ public class SortDataRows {
 
     // create new sort temp directory
     CarbonDataProcessorUtil.createLocations(parameters.getTempFileLocation());
-    this.dataSorterAndWriterExecutorService =
-        Executors.newFixedThreadPool(parameters.getNumberOfCores());
+    this.dataSorterAndWriterExecutorService = Executors
+        .newFixedThreadPool(parameters.getNumberOfCores(),
+            new CarbonThreadFactory("SortDataRowPool:" + parameters.getTableName()));
     semaphore = new Semaphore(parameters.getNumberOfCores());
   }
 
@@ -128,9 +130,9 @@ public class SortDataRows {
         semaphore.acquire();
         dataSorterAndWriterExecutorService.execute(new DataSorterAndWriter(recordHolderListLocal));
       } catch (InterruptedException e) {
-        LOGGER.error(
-            "exception occurred while trying to acquire a semaphore lock: " + e.getMessage());
-        throw new CarbonSortKeyAndGroupByException(e.getMessage());
+        LOGGER.error(e,
+            "exception occurred while trying to acquire a semaphore lock: ");
+        throw new CarbonSortKeyAndGroupByException(e);
       }
       // create the new holder Array
       this.recordHolderList = new Object[this.sortBufferSize][];
@@ -379,14 +381,21 @@ public class SortDataRows {
      * @throws CarbonSortKeyAndGroupByException
      */
     public void notifyFailed(Throwable exception) throws CarbonSortKeyAndGroupByException {
-      dataSorterAndWriterExecutorService.shutdownNow();
-      intermediateFileMerger.close();
+      close();
       parameters.getObserver().setFailed(true);
       LOGGER.error(exception);
       throw new CarbonSortKeyAndGroupByException(exception);
     }
   }
 
+  public void close() {
+    if (null != dataSorterAndWriterExecutorService && !dataSorterAndWriterExecutorService
+        .isShutdown()) {
+      dataSorterAndWriterExecutorService.shutdownNow();
+    }
+    intermediateFileMerger.close();
+  }
+
   /**
    * This class is responsible for sorting and writing the object
    * array which holds the records equal to given array size

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 d234ce2..9c995a5 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
@@ -20,13 +20,16 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
+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;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.processing.sort.exception.CarbonSortKeyAndGroupByException;
 
 /**
@@ -50,11 +53,15 @@ public class SortIntermediateFileMerger {
 
   private final Object lockObject = new Object();
 
+  private List<Future<Void>> mergerTask;
+
   public SortIntermediateFileMerger(SortParameters parameters) {
     this.parameters = parameters;
     // processed file list
     this.procFiles = new ArrayList<File>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    this.executorService = Executors.newFixedThreadPool(parameters.getNumberOfCores());
+    this.executorService = Executors.newFixedThreadPool(parameters.getNumberOfCores(),
+        new CarbonThreadFactory("SafeIntermediateMergerPool:" + parameters.getTableName()));
+    mergerTask = new ArrayList<>();
   }
 
   public void addFileToMerge(File sortTempFile) {
@@ -91,7 +98,7 @@ public class SortIntermediateFileMerger {
         chosenTempDir + File.separator + parameters.getTableName() + System
             .nanoTime() + CarbonCommonConstants.MERGERD_EXTENSION);
     IntermediateFileMerger merger = new IntermediateFileMerger(parameters, intermediateFiles, file);
-    executorService.execute(merger);
+    mergerTask.add(executorService.submit(merger));
   }
 
   public void finish() throws CarbonSortKeyAndGroupByException {
@@ -103,10 +110,22 @@ public class SortIntermediateFileMerger {
     }
     procFiles.clear();
     procFiles = null;
+    checkForFailure();
+  }
+
+  private void checkForFailure() throws CarbonSortKeyAndGroupByException {
+    for (int i = 0; i < mergerTask.size(); i++) {
+      try {
+        mergerTask.get(i).get();
+      } catch (InterruptedException | ExecutionException e) {
+        LOGGER.error(e, e.getMessage());
+        throw new CarbonSortKeyAndGroupByException(e);
+      }
+    }
   }
 
   public void close() {
-    if (executorService.isShutdown()) {
+    if (!executorService.isShutdown()) {
       executorService.shutdownNow();
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 c4b0b31..3e56605 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
@@ -34,6 +34,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.metadata.datatype.DataType;
 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;
@@ -153,7 +154,8 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
    */
   public SortTempFileChunkHolder(File tempFile, int dimensionCount, int complexDimensionCount,
       int measureCount, int fileBufferSize, int noDictionaryCount, DataType[] aggType,
-      boolean[] isNoDictionaryDimensionColumn, boolean[] isNoDictionarySortColumn) {
+      boolean[] isNoDictionaryDimensionColumn, boolean[] isNoDictionarySortColumn,
+      String tableName) {
     // set temp file
     this.tempFile = tempFile;
 
@@ -165,7 +167,8 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
     this.noDictionaryCount = noDictionaryCount;
     // set mdkey length
     this.fileBufferSize = fileBufferSize;
-    this.executorService = Executors.newFixedThreadPool(1);
+    this.executorService = Executors
+        .newFixedThreadPool(1, new CarbonThreadFactory("SafeSortTempChunkHolderPool:" + tableName));
     this.aggType = aggType;
 
     this.isNoDictionaryDimensionColumn = isNoDictionaryDimensionColumn;
@@ -407,7 +410,9 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
    */
   public void closeStream() {
     CarbonUtil.closeStreams(stream);
-    executorService.shutdown();
+    if (null != executorService) {
+      executorService.shutdownNow();
+    }
     this.backupBuffer = null;
     this.currentBuffer = null;
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 2c275bf..78f1637 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
@@ -48,6 +48,7 @@ import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 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.datatypes.GenericDataType;
 import org.apache.carbondata.processing.loading.sort.SortScopeOptions;
@@ -238,11 +239,13 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     }
 
     blockletProcessingCount = new AtomicInteger(0);
-    producerExecutorService = Executors.newFixedThreadPool(numberOfCores);
+    producerExecutorService = Executors.newFixedThreadPool(numberOfCores,
+        new CarbonThreadFactory("ProducerPool:" + model.getTableName()));
     producerExecutorServiceTaskList =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     LOGGER.info("Initializing writer executors");
-    consumerExecutorService = Executors.newFixedThreadPool(1);
+    consumerExecutorService = Executors
+        .newFixedThreadPool(1, new CarbonThreadFactory("ConsumerPool:" + model.getTableName()));
     consumerExecutorServiceTaskList = new ArrayList<>(1);
     semaphore = new Semaphore(numberOfCores);
     tablePageList = new TablePageList();
@@ -357,12 +360,20 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   public void finish() throws CarbonDataWriterException {
     // still some data is present in stores if entryCount is more
     // than 0
+    if (null == dataWriter) {
+      return;
+    }
+    if (producerExecutorService.isShutdown()) {
+      return;
+    }
+    LOGGER.info("Started Finish Operation");
     try {
       semaphore.acquire();
       producerExecutorServiceTaskList.add(producerExecutorService
           .submit(new Producer(tablePageList, dataRows, ++writerTaskSequenceCounter, true)));
       blockletProcessingCount.incrementAndGet();
       processedDataCount += entryCount;
+      LOGGER.info("Total Number Of records added to store: " + processedDataCount);
       closeWriterExecutionService(producerExecutorService);
       processWriteTaskSubmitList(producerExecutorServiceTaskList);
       processingComplete = true;
@@ -666,6 +677,10 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       return tablePage;
     }
 
+    /**
+     * @param tablePage
+     * @param index
+     */
     public synchronized void put(TablePage tablePage, int index) {
       tablePages[index] = tablePage;
       // notify the consumer thread when index at which object is to be inserted

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 acb3b3b..972e414 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
@@ -52,6 +52,7 @@ 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;
+import org.apache.carbondata.core.util.CarbonThreadFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonIndexFileWriter;
@@ -174,7 +175,8 @@ public abstract class AbstractFactDataWriter implements CarbonFactDataWriter {
     LOGGER.info("Total file size: " + fileSizeInBytes + " and dataBlock Size: " +
         blockSizeThreshold);
 
-    this.executorService = Executors.newFixedThreadPool(1);
+    this.executorService = Executors.newFixedThreadPool(1,
+        new CarbonThreadFactory("LocalToHDFSCopyPool:" + dataWriterVo.getTableName()));
     executorServiceSubmitList = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     // in case of compaction we will pass the cardinality.
     this.localCardinality = dataWriterVo.getColCardinality();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 70a8703..d8ae8ff 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
@@ -99,6 +99,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
       buffer.flip();
       channel.write(buffer);
     } catch (IOException e) {
+      LOGGER.error(e, "Problem while writing the carbon file");
       throw new CarbonDataWriterException("Problem while writing the carbon file: ", e);
     }
   }
@@ -184,7 +185,8 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
       }
       pageId = 0;
     } catch (IOException e) {
-      throw new CarbonDataWriterException("Problem when writing file", e);
+      LOGGER.error(e, "Problem while writing file");
+      throw new CarbonDataWriterException("Problem while writing file", e);
     }
     // clear the data holder
     blockletDataHolder.clear();
@@ -213,6 +215,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
         measureStartIndex++;
       }
     } catch (IOException e) {
+      LOGGER.error(e, "Problem while getting the data chunks");
       throw new CarbonDataWriterException("Problem while getting the data chunks", e);
     }
     return size;
@@ -346,6 +349,7 @@ public class CarbonFactDataWriterImplV3 extends AbstractFactDataWriter {
     try {
       writeIndexFile();
     } catch (IOException e) {
+      LOGGER.error(e, "Problem while writing the index file");
       throw new CarbonDataWriterException("Problem while writing the index file", e);
     }
     closeExecutorService();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a734add5/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 8681269..0b88684 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
@@ -67,6 +67,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 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;
@@ -216,7 +217,8 @@ public final class CarbonLoaderUtil {
       throw new RuntimeException("Store location not set for the key " + tempLocationKey);
     }
     // submit local folder clean up in another thread so that main thread execution is not blocked
-    ExecutorService localFolderDeletionService = Executors.newFixedThreadPool(1);
+    ExecutorService localFolderDeletionService = Executors
+        .newFixedThreadPool(1, new CarbonThreadFactory("LocalFolderDeletionPool:" + tableName));
     try {
       localFolderDeletionService.submit(new Callable<Void>() {
         @Override public Void call() throws Exception {


[38/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/DataField.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataField.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataField.java
new file mode 100644
index 0000000..5c81bb9
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataField.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.processing.loading;
+
+import java.io.Serializable;
+
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+
+/**
+ * Metadata class for each column of table.
+ */
+public class DataField implements Serializable {
+
+  public DataField(CarbonColumn column) {
+    this.column = column;
+  }
+
+  private CarbonColumn column;
+
+  private String dateFormat;
+
+  public boolean hasDictionaryEncoding() {
+    return column.hasEncoding(Encoding.DICTIONARY);
+  }
+
+  public CarbonColumn getColumn() {
+    return column;
+  }
+
+  public String getDateFormat() {
+    return dateFormat;
+  }
+
+  public void setDateFormat(String dateFormat) {
+    this.dateFormat = dateFormat;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadExecutor.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadExecutor.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadExecutor.java
new file mode 100644
index 0000000..10b19b7
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadExecutor.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+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.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.processing.loading.exception.BadRecordFoundException;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.exception.NoRetryException;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+
+/**
+ * It executes the data load.
+ */
+public class DataLoadExecutor {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DataLoadExecutor.class.getName());
+
+  private AbstractDataLoadProcessorStep loadProcessorStep;
+
+  private boolean isClosed;
+
+  public void execute(CarbonLoadModel loadModel, String[] storeLocation,
+      CarbonIterator<Object[]>[] inputIterators) throws Exception {
+    try {
+      loadProcessorStep =
+          new DataLoadProcessBuilder().build(loadModel, storeLocation, inputIterators);
+      // 1. initialize
+      loadProcessorStep.initialize();
+      LOGGER.info("Data Loading is started for table " + loadModel.getTableName());
+      // 2. execute the step
+      loadProcessorStep.execute();
+      // check and remove any bad record key from bad record entry logger static map
+      if (badRecordFound(
+          loadModel.getCarbonDataLoadSchema().getCarbonTable().getCarbonTableIdentifier())) {
+        LOGGER.error("Data Load is partially success for table " + loadModel.getTableName());
+      } else {
+        LOGGER.info("Data loading is successful for table " + loadModel.getTableName());
+      }
+    } catch (CarbonDataLoadingException e) {
+      if (e instanceof BadRecordFoundException) {
+        throw new NoRetryException(e.getMessage());
+      } else {
+        throw e;
+      }
+    } catch (Exception e) {
+      LOGGER.error(e, "Data Loading failed for table " + loadModel.getTableName());
+      throw new CarbonDataLoadingException(
+          "Data Loading failed for table " + loadModel.getTableName(), e);
+    } finally {
+      removeBadRecordKey(
+          loadModel.getCarbonDataLoadSchema().getCarbonTable().getCarbonTableIdentifier());
+    }
+  }
+
+  /**
+   * This method will remove any bad record key from the map entry
+   *
+   * @param carbonTableIdentifier
+   * @return
+   */
+  private boolean badRecordFound(CarbonTableIdentifier carbonTableIdentifier) {
+    String badRecordLoggerKey = carbonTableIdentifier.getBadRecordLoggerKey();
+    boolean badRecordKeyFound = false;
+    if (null != BadRecordsLogger.hasBadRecord(badRecordLoggerKey)) {
+      badRecordKeyFound = true;
+    }
+    return badRecordKeyFound;
+  }
+
+  /**
+   * This method will remove the bad record key from bad record logger
+   *
+   * @param carbonTableIdentifier
+   */
+  private void removeBadRecordKey(CarbonTableIdentifier carbonTableIdentifier) {
+    String badRecordLoggerKey = carbonTableIdentifier.getBadRecordLoggerKey();
+    BadRecordsLogger.removeBadRecordKey(badRecordLoggerKey);
+  }
+
+  /**
+   * Method to clean all the resource
+   */
+  public void close() {
+    if (!isClosed && loadProcessorStep != null) {
+      loadProcessorStep.close();
+    }
+    isClosed = true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..05104a2
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.CarbonIterator;
+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.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.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.InputProcessorStepImpl;
+import org.apache.carbondata.processing.loading.steps.SortProcessorStepImpl;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * It builds the pipe line of steps for loading data to carbon.
+ */
+public final class DataLoadProcessBuilder {
+
+  public AbstractDataLoadProcessorStep build(CarbonLoadModel loadModel, String[] storeLocation,
+      CarbonIterator[] inputIterators) throws Exception {
+    CarbonDataLoadConfiguration configuration = createConfiguration(loadModel, storeLocation);
+    SortScopeOptions.SortScope sortScope = CarbonDataProcessorUtil.getSortScope(configuration);
+    if (!configuration.isSortTable() || sortScope.equals(SortScopeOptions.SortScope.NO_SORT)) {
+      return buildInternalForNoSort(inputIterators, configuration);
+    } else if (configuration.getBucketingInfo() != null) {
+      return buildInternalForBucketing(inputIterators, configuration);
+    } else if (sortScope.equals(SortScopeOptions.SortScope.BATCH_SORT)) {
+      return buildInternalForBatchSort(inputIterators, configuration);
+    } else {
+      return buildInternal(inputIterators, configuration);
+    }
+  }
+
+  private AbstractDataLoadProcessorStep buildInternal(CarbonIterator[] inputIterators,
+      CarbonDataLoadConfiguration configuration) {
+    // 1. Reads the data input iterators and parses the data.
+    AbstractDataLoadProcessorStep inputProcessorStep =
+        new InputProcessorStepImpl(configuration, inputIterators);
+    // 2. Converts the data like dictionary or non dictionary or complex objects depends on
+    // data types and configurations.
+    AbstractDataLoadProcessorStep converterProcessorStep =
+        new DataConverterProcessorStepImpl(configuration, inputProcessorStep);
+    // 3. Sorts the data by SortColumn
+    AbstractDataLoadProcessorStep sortProcessorStep =
+        new SortProcessorStepImpl(configuration, converterProcessorStep);
+    // 4. Writes the sorted data in carbondata format.
+    return new DataWriterProcessorStepImpl(configuration, sortProcessorStep);
+  }
+
+  private AbstractDataLoadProcessorStep buildInternalForNoSort(CarbonIterator[] inputIterators,
+      CarbonDataLoadConfiguration configuration) {
+    // 1. Reads the data input iterators and parses the data.
+    AbstractDataLoadProcessorStep inputProcessorStep =
+        new InputProcessorStepImpl(configuration, inputIterators);
+    // 2. Converts the data like dictionary or non dictionary or complex objects depends on
+    // data types and configurations.
+    AbstractDataLoadProcessorStep converterProcessorStep =
+        new DataConverterProcessorStepImpl(configuration, inputProcessorStep);
+    // 3. Writes the sorted data in carbondata format.
+    AbstractDataLoadProcessorStep writerProcessorStep =
+        new CarbonRowDataWriterProcessorStepImpl(configuration, converterProcessorStep);
+    return writerProcessorStep;
+  }
+
+  private AbstractDataLoadProcessorStep buildInternalForBatchSort(CarbonIterator[] inputIterators,
+      CarbonDataLoadConfiguration configuration) {
+    // 1. Reads the data input iterators and parses the data.
+    AbstractDataLoadProcessorStep inputProcessorStep =
+        new InputProcessorStepImpl(configuration, inputIterators);
+    // 2. Converts the data like dictionary or non dictionary or complex objects depends on
+    // data types and configurations.
+    AbstractDataLoadProcessorStep converterProcessorStep =
+        new DataConverterProcessorStepImpl(configuration, inputProcessorStep);
+    // 3. Sorts the data by SortColumn or not
+    AbstractDataLoadProcessorStep sortProcessorStep =
+        new SortProcessorStepImpl(configuration, converterProcessorStep);
+    // 4. Writes the sorted data in carbondata format.
+    return new DataWriterBatchProcessorStepImpl(configuration, sortProcessorStep);
+  }
+
+  private AbstractDataLoadProcessorStep buildInternalForBucketing(CarbonIterator[] inputIterators,
+      CarbonDataLoadConfiguration configuration) throws Exception {
+    // 1. Reads the data input iterators and parses the data.
+    AbstractDataLoadProcessorStep inputProcessorStep =
+        new InputProcessorStepImpl(configuration, inputIterators);
+    // 2. Converts the data like dictionary or non dictionary or complex objects depends on
+    // data types and configurations.
+    AbstractDataLoadProcessorStep converterProcessorStep =
+        new DataConverterProcessorWithBucketingStepImpl(configuration, inputProcessorStep);
+    // 3. Sorts the data by SortColumn or not
+    AbstractDataLoadProcessorStep sortProcessorStep =
+        new SortProcessorStepImpl(configuration, converterProcessorStep);
+    // 4. Writes the sorted data in carbondata format.
+    return new DataWriterProcessorStepImpl(configuration, sortProcessorStep);
+  }
+
+  public static CarbonDataLoadConfiguration createConfiguration(CarbonLoadModel loadModel,
+      String[] storeLocation) {
+    CarbonDataProcessorUtil.createLocations(storeLocation);
+
+    String databaseName = loadModel.getDatabaseName();
+    String tableName = loadModel.getTableName();
+    String tempLocationKey = CarbonDataProcessorUtil
+        .getTempStoreLocationKey(databaseName, tableName, loadModel.getSegmentId(),
+            loadModel.getTaskNo(), false, false);
+    CarbonProperties.getInstance().addProperty(tempLocationKey,
+        StringUtils.join(storeLocation, File.pathSeparator));
+    CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS, loadModel.getStorePath());
+
+    return createConfiguration(loadModel);
+  }
+
+  public static CarbonDataLoadConfiguration createConfiguration(CarbonLoadModel loadModel) {
+    CarbonDataLoadConfiguration configuration = new CarbonDataLoadConfiguration();
+    CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
+    AbsoluteTableIdentifier identifier = carbonTable.getAbsoluteTableIdentifier();
+    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,
+        new String[] { loadModel.getComplexDelimiterLevel1(),
+            loadModel.getComplexDelimiterLevel2() });
+    configuration.setDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT,
+        loadModel.getSerializationNullFormat().split(",")[1]);
+    configuration.setDataLoadProperty(DataLoadProcessorConstants.FACT_TIME_STAMP,
+        loadModel.getFactTimeStamp());
+    configuration.setDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ENABLE,
+        loadModel.getBadRecordsLoggerEnable().split(",")[1]);
+    configuration.setDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ACTION,
+        loadModel.getBadRecordsAction().split(",")[1]);
+    configuration.setDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD,
+        loadModel.getIsEmptyDataBadRecord().split(",")[1]);
+    configuration.setDataLoadProperty(DataLoadProcessorConstants.FACT_FILE_PATH,
+        loadModel.getFactFilePath());
+    configuration
+        .setDataLoadProperty(CarbonCommonConstants.LOAD_SORT_SCOPE, loadModel.getSortScope());
+    configuration.setDataLoadProperty(CarbonCommonConstants.LOAD_BATCH_SORT_SIZE_INMB,
+        loadModel.getBatchSortSizeInMb());
+    configuration.setDataLoadProperty(CarbonCommonConstants.LOAD_GLOBAL_SORT_PARTITIONS,
+        loadModel.getGlobalSortPartitions());
+    configuration.setDataLoadProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH,
+        loadModel.getBadRecordsLocation());
+    CarbonMetadata.getInstance().addCarbonTable(carbonTable);
+    List<CarbonDimension> dimensions =
+        carbonTable.getDimensionByTableName(carbonTable.getFactTableName());
+    List<CarbonMeasure> measures =
+        carbonTable.getMeasureByTableName(carbonTable.getFactTableName());
+    Map<String, String> dateFormatMap =
+        CarbonDataProcessorUtil.getDateFormatMap(loadModel.getDateFormat());
+    List<DataField> dataFields = new ArrayList<>();
+    List<DataField> complexDataFields = new ArrayList<>();
+
+    // First add dictionary and non dictionary dimensions because these are part of mdk key.
+    // And then add complex data types and measures.
+    for (CarbonColumn column : dimensions) {
+      DataField dataField = new DataField(column);
+      dataField.setDateFormat(dateFormatMap.get(column.getColName()));
+      if (column.isComplex()) {
+        complexDataFields.add(dataField);
+      } else {
+        dataFields.add(dataField);
+      }
+    }
+    dataFields.addAll(complexDataFields);
+    for (CarbonColumn column : measures) {
+      // This dummy measure is added when no measure was present. We no need to load it.
+      if (!(column.getColName().equals("default_dummy_measure"))) {
+        dataFields.add(new DataField(column));
+      }
+    }
+    configuration.setDataFields(dataFields.toArray(new DataField[dataFields.size()]));
+    configuration.setBucketingInfo(carbonTable.getBucketingInfo(carbonTable.getFactTableName()));
+    // configuration for one pass load: dictionary server info
+    configuration.setUseOnePass(loadModel.getUseOnePass());
+    configuration.setDictionaryServerHost(loadModel.getDictionaryServerHost());
+    configuration.setDictionaryServerPort(loadModel.getDictionaryServerPort());
+    configuration.setPreFetch(loadModel.isPreFetch());
+    configuration.setNumberOfSortColumns(carbonTable.getNumberOfSortColumns());
+    configuration.setNumberOfNoDictSortColumns(carbonTable.getNumberOfNoDictSortColumns());
+
+    TableSpec tableSpec = new TableSpec(dimensions, measures);
+    configuration.setTableSpec(tableSpec);
+    return configuration;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/FailureCauses.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/FailureCauses.java b/processing/src/main/java/org/apache/carbondata/processing/loading/FailureCauses.java
new file mode 100644
index 0000000..6e5f91a
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/FailureCauses.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading;
+
+/**
+ * This Enum is used to determine the Reasons of Failure.
+ */
+public enum FailureCauses {
+  NONE,
+  BAD_RECORDS,
+  EXECUTOR_FAILURE,
+  STATUS_FILE_UPDATION_FAILURE,
+  MULTIPLE_INPUT_ROWS_MATCHING
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/complexobjects/ArrayObject.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/complexobjects/ArrayObject.java b/processing/src/main/java/org/apache/carbondata/processing/loading/complexobjects/ArrayObject.java
new file mode 100644
index 0000000..ecd46cc
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/complexobjects/ArrayObject.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.complexobjects;
+
+public class ArrayObject {
+
+  private Object[] data;
+
+  public ArrayObject(Object[] data) {
+    this.data = data;
+  }
+
+  public Object[] getData() {
+    return data;
+  }
+
+  public void setData(Object[] data) {
+    this.data = data;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/complexobjects/StructObject.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/complexobjects/StructObject.java b/processing/src/main/java/org/apache/carbondata/processing/loading/complexobjects/StructObject.java
new file mode 100644
index 0000000..c026a48
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/complexobjects/StructObject.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.processing.loading.complexobjects;
+
+public class StructObject {
+
+  private Object[] data;
+
+  public StructObject(Object[] data) {
+    this.data = data;
+  }
+
+  public Object[] getData() {
+    return data;
+  }
+
+  public void setData(Object[] data) {
+    this.data = data;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/constants/DataLoadProcessorConstants.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/constants/DataLoadProcessorConstants.java b/processing/src/main/java/org/apache/carbondata/processing/loading/constants/DataLoadProcessorConstants.java
new file mode 100644
index 0000000..260661b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/constants/DataLoadProcessorConstants.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.processing.loading.constants;
+
+/**
+ * Constants used in data loading.
+ */
+public final class DataLoadProcessorConstants {
+
+  public static final String FACT_TIME_STAMP = "FACT_TIME_STAMP";
+
+  public static final String COMPLEX_DELIMITERS = "COMPLEX_DELIMITERS";
+
+  public static final String SERIALIZATION_NULL_FORMAT = "SERIALIZATION_NULL_FORMAT";
+
+  public static final String BAD_RECORDS_LOGGER_ENABLE = "BAD_RECORDS_LOGGER_ENABLE";
+
+  public static final String BAD_RECORDS_LOGGER_ACTION = "BAD_RECORDS_LOGGER_ACTION";
+
+  public static final String IS_EMPTY_DATA_BAD_RECORD = "IS_EMPTY_DATA_BAD_RECORD";
+
+  public static final String FACT_FILE_PATH = "FACT_FILE_PATH";
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/BadRecordLogHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/BadRecordLogHolder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/BadRecordLogHolder.java
new file mode 100644
index 0000000..aeb4d15
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/BadRecordLogHolder.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.processing.loading.converter;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * It is holder for reason of bad records.
+ */
+public class BadRecordLogHolder {
+
+  /**
+   * this map will hold the bad record unified message for columns
+   */
+  private Map<String, String> columnMessageMap = new HashMap<>();
+
+  private String reason;
+
+  private boolean badRecordAdded;
+
+  private boolean isLogged;
+
+  public String getReason() {
+    return reason;
+  }
+
+  public void setReason(String reason) {
+    this.reason = reason;
+    badRecordAdded = true;
+  }
+
+  public boolean isBadRecordNotAdded() {
+    return badRecordAdded;
+  }
+
+  public void clear() {
+    this.badRecordAdded = false;
+  }
+
+  public boolean isLogged() {
+    return isLogged;
+  }
+
+  public void setLogged(boolean logged) {
+    isLogged = logged;
+  }
+
+  public Map<String, String> getColumnMessageMap() {
+    return columnMessageMap;
+  }
+
+  /**
+   * this method will clear the map entries
+   */
+  public void finish() {
+    if (null != columnMessageMap) {
+      columnMessageMap.clear();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/DictionaryCardinalityFinder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/DictionaryCardinalityFinder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/DictionaryCardinalityFinder.java
new file mode 100644
index 0000000..aa84fc3
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/DictionaryCardinalityFinder.java
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.converter;
+
+/**
+ * Finds the current cardinality of dimensions.
+ */
+public interface DictionaryCardinalityFinder {
+
+  int[] getCardinality();
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/FieldConverter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/FieldConverter.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/FieldConverter.java
new file mode 100644
index 0000000..8a3e2eb
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/FieldConverter.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.processing.loading.converter;
+
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+
+/**
+ * This interface converts/transforms the column field.
+ */
+public interface FieldConverter {
+
+  /**
+   * It converts the column field and updates the data in same location/index in row.
+   * @param row
+   * @return the status whether it could be loaded or not, usually when record is added
+   * to bad records then it returns false.
+   * @throws CarbonDataLoadingException
+   */
+  void convert(CarbonRow row, BadRecordLogHolder logHolder) throws CarbonDataLoadingException;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..fd3a650
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/RowConverter.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.processing.loading.converter;
+
+import java.io.IOException;
+
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+
+/**
+ * convert the row
+ */
+public interface RowConverter extends DictionaryCardinalityFinder {
+
+  void initialize() throws IOException;
+
+  CarbonRow convert(CarbonRow row) throws CarbonDataLoadingException;
+
+  RowConverter createCopyForNewThread();
+
+  void finish();
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/AbstractDictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/AbstractDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/AbstractDictionaryFieldConverterImpl.java
new file mode 100644
index 0000000..5349e33
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/AbstractDictionaryFieldConverterImpl.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.converter.impl;
+
+import java.util.List;
+
+import org.apache.carbondata.processing.loading.converter.FieldConverter;
+
+public abstract class AbstractDictionaryFieldConverterImpl implements FieldConverter {
+
+  public abstract void fillColumnCardinality(List<Integer> cardinality);
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.java
new file mode 100644
index 0000000..5ac832d
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/ComplexFieldConverterImpl.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.processing.loading.converter.impl;
+
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.util.List;
+
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.processing.datatypes.GenericDataType;
+import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+
+public class ComplexFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
+
+  private GenericDataType genericDataType;
+
+  private int index;
+
+  public ComplexFieldConverterImpl(GenericDataType genericDataType, int index) {
+    this.genericDataType = genericDataType;
+    this.index = index;
+  }
+
+  @Override
+  public void convert(CarbonRow row, BadRecordLogHolder logHolder) {
+    Object object = row.getObject(index);
+    // TODO Its temporary, needs refactor here.
+    ByteArrayOutputStream byteArray = new ByteArrayOutputStream();
+    DataOutputStream dataOutputStream = new DataOutputStream(byteArray);
+    try {
+      genericDataType.writeByteArray(object, dataOutputStream);
+      dataOutputStream.close();
+      row.update(byteArray.toByteArray(), index);
+    } catch (Exception e) {
+      throw new CarbonDataLoadingException(object + "", e);
+    }
+  }
+
+  @Override public void fillColumnCardinality(List<Integer> cardinality) {
+    genericDataType.fillCardinality(cardinality);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
new file mode 100644
index 0000000..2671393
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DictionaryFieldConverterImpl.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.converter.impl;
+
+import java.io.IOException;
+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.cache.Cache;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.devapi.BiDictionary;
+import org.apache.carbondata.core.devapi.DictionaryGenerationException;
+import org.apache.carbondata.core.dictionary.client.DictionaryClient;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
+import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessageType;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+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.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
+import org.apache.carbondata.processing.loading.dictionary.DictionaryServerClientDictionary;
+import org.apache.carbondata.processing.loading.dictionary.PreCreatedDictionary;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+public class DictionaryFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DictionaryFieldConverterImpl.class.getName());
+
+  private BiDictionary<Integer, Object> dictionaryGenerator;
+
+  private int index;
+
+  private CarbonDimension carbonDimension;
+
+  private String nullFormat;
+
+  private Dictionary dictionary;
+
+  private DictionaryMessage dictionaryMessage;
+
+  private boolean isEmptyBadRecord;
+
+  public DictionaryFieldConverterImpl(DataField dataField,
+      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
+      CarbonTableIdentifier carbonTableIdentifier, String nullFormat, int index,
+      DictionaryClient client, boolean useOnePass, String storePath,
+      Map<Object, Integer> localCache, boolean isEmptyBadRecord) throws IOException {
+    this.index = index;
+    this.carbonDimension = (CarbonDimension) dataField.getColumn();
+    this.nullFormat = nullFormat;
+    this.isEmptyBadRecord = isEmptyBadRecord;
+    DictionaryColumnUniqueIdentifier identifier =
+        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+            dataField.getColumn().getColumnIdentifier(), dataField.getColumn().getDataType(),
+            CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier));
+
+    // if use one pass, use DictionaryServerClientDictionary
+    if (useOnePass) {
+      if (CarbonUtil.isFileExistsForGivenColumn(storePath, identifier)) {
+        dictionary = cache.get(identifier);
+      }
+      dictionaryMessage = new DictionaryMessage();
+      dictionaryMessage.setColumnName(dataField.getColumn().getColName());
+      // for table initialization
+      dictionaryMessage.setTableUniqueId(carbonTableIdentifier.getTableId());
+      dictionaryMessage.setData("0");
+      // for generate dictionary
+      dictionaryMessage.setType(DictionaryMessageType.DICT_GENERATION);
+      dictionaryGenerator = new DictionaryServerClientDictionary(dictionary, client,
+          dictionaryMessage, localCache);
+    } else {
+      dictionary = cache.get(identifier);
+      dictionaryGenerator = new PreCreatedDictionary(dictionary);
+    }
+  }
+
+  @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder)
+      throws CarbonDataLoadingException {
+    try {
+      String parsedValue = null;
+      String dimensionValue = row.getString(index);
+      if (dimensionValue == null || dimensionValue.equals(nullFormat)) {
+        parsedValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
+      } else {
+        parsedValue = DataTypeUtil.parseValue(dimensionValue, carbonDimension);
+      }
+      if (null == parsedValue) {
+        if ((dimensionValue.length() > 0) || (dimensionValue.length() == 0 && isEmptyBadRecord)) {
+          String message = logHolder.getColumnMessageMap().get(carbonDimension.getColName());
+          if (null == message) {
+            message = CarbonDataProcessorUtil.prepareFailureReason(
+                carbonDimension.getColName(), carbonDimension.getDataType());
+            logHolder.getColumnMessageMap().put(carbonDimension.getColName(), message);
+          } logHolder.setReason(message);
+        }
+        row.update(CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY, index);
+      } else {
+        row.update(dictionaryGenerator.getOrGenerateKey(parsedValue), index);
+      }
+    } catch (DictionaryGenerationException e) {
+      throw new CarbonDataLoadingException(e);
+    }
+  }
+
+  @Override
+  public void fillColumnCardinality(List<Integer> cardinality) {
+    cardinality.add(dictionaryGenerator.size());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DirectDictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DirectDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DirectDictionaryFieldConverterImpl.java
new file mode 100644
index 0000000..24c2f00
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/DirectDictionaryFieldConverterImpl.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.converter.impl;
+
+import java.util.List;
+
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.converter.BadRecordLogHolder;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+public class DirectDictionaryFieldConverterImpl extends AbstractDictionaryFieldConverterImpl {
+
+  private DirectDictionaryGenerator directDictionaryGenerator;
+
+  private int index;
+
+  private String nullFormat;
+
+  private CarbonColumn column;
+  private boolean isEmptyBadRecord;
+
+  public DirectDictionaryFieldConverterImpl(DataField dataField, String nullFormat, int index,
+      boolean isEmptyBadRecord) {
+    this.nullFormat = nullFormat;
+    this.column = dataField.getColumn();
+    if (dataField.getDateFormat() != null && !dataField.getDateFormat().isEmpty()) {
+      this.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+          .getDirectDictionaryGenerator(dataField.getColumn().getDataType(),
+              dataField.getDateFormat());
+
+    } else {
+      this.directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+          .getDirectDictionaryGenerator(dataField.getColumn().getDataType());
+    }
+    this.index = index;
+    this.isEmptyBadRecord = isEmptyBadRecord;
+  }
+
+  @Override
+  public void convert(CarbonRow row, BadRecordLogHolder logHolder) {
+    String value = row.getString(index);
+    if (value == null) {
+      logHolder.setReason(
+          CarbonDataProcessorUtil.prepareFailureReason(column.getColName(), column.getDataType()));
+      row.update(1, index);
+    } else if (value.equals(nullFormat)) {
+      row.update(1, index);
+    } else {
+      int key = directDictionaryGenerator.generateDirectSurrogateKey(value);
+      if (key == 1) {
+        if ((value.length() > 0) || (value.length() == 0 && isEmptyBadRecord)) {
+          String message = logHolder.getColumnMessageMap().get(column.getColName());
+          if (null == message) {
+            message = CarbonDataProcessorUtil.prepareFailureReason(
+                column.getColName(), column.getDataType());
+            logHolder.getColumnMessageMap().put(column.getColName(), message);
+          }
+          logHolder.setReason(message);
+        }
+      }
+      row.update(key, index);
+    }
+  }
+
+  @Override
+  public void fillColumnCardinality(List<Integer> cardinality) {
+    cardinality.add(Integer.MAX_VALUE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..2efbe26
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/FieldEncoderFactory.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.converter.impl;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.dictionary.client.DictionaryClient;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.processing.datatypes.ArrayDataType;
+import org.apache.carbondata.processing.datatypes.GenericDataType;
+import org.apache.carbondata.processing.datatypes.PrimitiveDataType;
+import org.apache.carbondata.processing.datatypes.StructDataType;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.converter.FieldConverter;
+
+public class FieldEncoderFactory {
+
+  private static FieldEncoderFactory instance;
+
+  private FieldEncoderFactory() {
+
+  }
+
+  public static FieldEncoderFactory getInstance() {
+    if (instance == null) {
+      instance = new FieldEncoderFactory();
+    }
+    return instance;
+  }
+
+  /**
+   * Creates the FieldConverter for all dimensions, for measures return null.
+   *
+   * @param dataField             column schema
+   * @param cache                 dicionary cache.
+   * @param carbonTableIdentifier table identifier
+   * @param index                 index of column in the row.
+   * @param isEmptyBadRecord
+   * @return
+   */
+  public FieldConverter createFieldEncoder(DataField dataField,
+      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
+      CarbonTableIdentifier carbonTableIdentifier, int index, String nullFormat,
+      DictionaryClient client, Boolean useOnePass, String storePath,
+      Map<Object, Integer> localCache, boolean isEmptyBadRecord)
+      throws IOException {
+    // Converters are only needed for dimensions and measures it return null.
+    if (dataField.getColumn().isDimension()) {
+      if (dataField.getColumn().hasEncoding(Encoding.DIRECT_DICTIONARY) &&
+          !dataField.getColumn().isComplex()) {
+        return new DirectDictionaryFieldConverterImpl(dataField, nullFormat, index,
+            isEmptyBadRecord);
+      } else if (dataField.getColumn().hasEncoding(Encoding.DICTIONARY) &&
+          !dataField.getColumn().isComplex()) {
+        return new DictionaryFieldConverterImpl(dataField, cache, carbonTableIdentifier, nullFormat,
+            index, client, useOnePass, storePath, localCache, isEmptyBadRecord);
+      } else if (dataField.getColumn().isComplex()) {
+        return new ComplexFieldConverterImpl(
+            createComplexType(dataField, cache, carbonTableIdentifier,
+                    client, useOnePass, storePath, localCache), index);
+      } else {
+        return new NonDictionaryFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord);
+      }
+    } else {
+      return new MeasureFieldConverterImpl(dataField, nullFormat, index, isEmptyBadRecord);
+    }
+  }
+
+  /**
+   * Create parser for the carbon column.
+   */
+  private static GenericDataType createComplexType(DataField dataField,
+      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
+      CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
+      String storePath, Map<Object, Integer> localCache) {
+    return createComplexType(dataField.getColumn(), dataField.getColumn().getColName(), cache,
+        carbonTableIdentifier, client, useOnePass, storePath, localCache);
+  }
+
+  /**
+   * This method may be called recursively if the carbon column is complex type.
+   *
+   * @return GenericDataType
+   */
+  private static GenericDataType createComplexType(CarbonColumn carbonColumn, String parentName,
+      Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache,
+      CarbonTableIdentifier carbonTableIdentifier, DictionaryClient client, Boolean useOnePass,
+      String storePath, Map<Object, Integer> localCache) {
+    switch (carbonColumn.getDataType()) {
+      case ARRAY:
+        List<CarbonDimension> listOfChildDimensions =
+            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
+        // Create array parser with complex delimiter
+        ArrayDataType arrayDataType =
+            new ArrayDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
+        for (CarbonDimension dimension : listOfChildDimensions) {
+          arrayDataType.addChildren(createComplexType(dimension, carbonColumn.getColName(), cache,
+              carbonTableIdentifier, client, useOnePass, storePath, localCache));
+        }
+        return arrayDataType;
+      case STRUCT:
+        List<CarbonDimension> dimensions =
+            ((CarbonDimension) carbonColumn).getListOfChildDimensions();
+        // Create struct parser with complex delimiter
+        StructDataType structDataType =
+            new StructDataType(carbonColumn.getColName(), parentName, carbonColumn.getColumnId());
+        for (CarbonDimension dimension : dimensions) {
+          structDataType.addChildren(createComplexType(dimension, carbonColumn.getColName(), cache,
+              carbonTableIdentifier, client, useOnePass, storePath, localCache));
+        }
+        return structDataType;
+      case MAP:
+        throw new UnsupportedOperationException("Complex type Map is not supported yet");
+      default:
+        return new PrimitiveDataType(carbonColumn.getColName(), parentName,
+            carbonColumn.getColumnId(), (CarbonDimension) carbonColumn, cache,
+            carbonTableIdentifier, client, useOnePass, storePath, localCache);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/MeasureFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/MeasureFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/MeasureFieldConverterImpl.java
new file mode 100644
index 0000000..06f7589
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/MeasureFieldConverterImpl.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.converter.impl;
+
+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.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
+import org.apache.carbondata.core.util.DataTypeUtil;
+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.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+/**
+ * Converter for measure
+ */
+public class MeasureFieldConverterImpl implements FieldConverter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(MeasureFieldConverterImpl.class.getName());
+
+  private int index;
+
+  private DataType dataType;
+
+  private CarbonMeasure measure;
+
+  private String nullformat;
+
+  private boolean isEmptyBadRecord;
+
+  public MeasureFieldConverterImpl(DataField dataField, String nullformat, int index,
+      boolean isEmptyBadRecord) {
+    this.dataType = dataField.getColumn().getDataType();
+    this.measure = (CarbonMeasure) dataField.getColumn();
+    this.nullformat = nullformat;
+    this.index = index;
+    this.isEmptyBadRecord = isEmptyBadRecord;
+  }
+
+  @Override
+  public void convert(CarbonRow row, BadRecordLogHolder logHolder)
+      throws CarbonDataLoadingException {
+    String value = row.getString(index);
+    Object output;
+    boolean isNull = CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(value);
+    if (value == null || isNull) {
+      String message = logHolder.getColumnMessageMap().get(measure.getColName());
+      if (null == message) {
+        message = CarbonDataProcessorUtil
+            .prepareFailureReason(measure.getColName(), measure.getDataType());
+        logHolder.getColumnMessageMap().put(measure.getColName(), message);
+      }
+      row.update(null, index);
+    } else if (value.length() == 0) {
+      if (isEmptyBadRecord) {
+        String message = logHolder.getColumnMessageMap().get(measure.getColName());
+        if (null == message) {
+          message = CarbonDataProcessorUtil
+              .prepareFailureReason(measure.getColName(), measure.getDataType());
+          logHolder.getColumnMessageMap().put(measure.getColName(), message);
+        }
+        logHolder.setReason(message);
+      }
+      row.update(null, index);
+    } else if (value.equals(nullformat)) {
+      row.update(null, index);
+    } else {
+      try {
+        output = DataTypeUtil.getMeasureValueBasedOnDataType(value, dataType, measure);
+        row.update(output, index);
+      } catch (NumberFormatException e) {
+        LOGGER.warn(
+            "Cant not convert value to Numeric type value. Value considered as null.");
+        logHolder.setReason(
+            CarbonDataProcessorUtil.prepareFailureReason(measure.getColName(), dataType));
+        output = null;
+        row.update(output, index);
+      }
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
new file mode 100644
index 0000000..8170680
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/NonDictionaryFieldConverterImpl.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.converter.impl;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.util.DataTypeUtil;
+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.util.CarbonDataProcessorUtil;
+
+public class NonDictionaryFieldConverterImpl implements FieldConverter {
+
+  private DataType dataType;
+
+  private int index;
+
+  private String nullformat;
+
+  private CarbonColumn column;
+
+  private boolean isEmptyBadRecord;
+
+  private DataField dataField;
+
+  public NonDictionaryFieldConverterImpl(DataField dataField, String nullformat, int index,
+      boolean isEmptyBadRecord) {
+    this.dataField = dataField;
+    this.dataType = dataField.getColumn().getDataType();
+    this.column = dataField.getColumn();
+    this.index = index;
+    this.nullformat = nullformat;
+    this.isEmptyBadRecord = isEmptyBadRecord;
+  }
+
+  @Override public void convert(CarbonRow row, BadRecordLogHolder logHolder) {
+    String dimensionValue = row.getString(index);
+    if (null == dimensionValue && column.getDataType() != DataType.STRING) {
+      logHolder.setReason(
+          CarbonDataProcessorUtil.prepareFailureReason(column.getColName(), column.getDataType()));
+      updateWithNullValue(row);
+    } else if (dimensionValue == null || dimensionValue.equals(nullformat)) {
+      updateWithNullValue(row);
+    } else {
+      try {
+        row.update(DataTypeUtil
+            .getBytesBasedOnDataTypeForNoDictionaryColumn(dimensionValue, dataType,
+                dataField.getDateFormat()), index);
+      } catch (Throwable ex) {
+        if (dimensionValue.length() > 0 || (dimensionValue.length() == 0 && isEmptyBadRecord)) {
+          String message = logHolder.getColumnMessageMap().get(column.getColName());
+          if (null == message) {
+            message = CarbonDataProcessorUtil
+                .prepareFailureReason(column.getColName(), column.getDataType());
+            logHolder.getColumnMessageMap().put(column.getColName(), message);
+          }
+          logHolder.setReason(message);
+          updateWithNullValue(row);
+        } else {
+          updateWithNullValue(row);
+        }
+      }
+    }
+  }
+
+  private void updateWithNullValue(CarbonRow row) {
+    if (dataType == DataType.STRING) {
+      row.update(CarbonCommonConstants.MEMBER_DEFAULT_VAL_ARRAY, index);
+    } else {
+      row.update(CarbonCommonConstants.EMPTY_BYTE_ARRAY, index);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..a4351ae
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/converter/impl/RowConverterImpl.java
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.converter.impl;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.cache.Cache;
+import org.apache.carbondata.core.cache.CacheProvider;
+import org.apache.carbondata.core.cache.CacheType;
+import org.apache.carbondata.core.cache.dictionary.Dictionary;
+import org.apache.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.dictionary.client.DictionaryClient;
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.processing.loading.BadRecordsLogger;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+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.exception.BadRecordFoundException;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+
+/**
+ * It converts the complete row if necessary, dictionary columns are encoded with dictionary values
+ * and nondictionary values are converted to binary.
+ */
+public class RowConverterImpl implements RowConverter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(RowConverterImpl.class.getName());
+
+  private CarbonDataLoadConfiguration configuration;
+
+  private DataField[] fields;
+
+  private FieldConverter[] fieldConverters;
+
+  private BadRecordsLogger badRecordLogger;
+
+  private BadRecordLogHolder logHolder;
+
+  private List<DictionaryClient> dictClients = new ArrayList<>();
+
+  private ExecutorService executorService;
+
+  private Cache<DictionaryColumnUniqueIdentifier, Dictionary> cache;
+
+  private Map<Object, Integer>[] localCaches;
+
+  public RowConverterImpl(DataField[] fields, CarbonDataLoadConfiguration configuration,
+      BadRecordsLogger badRecordLogger) {
+    this.fields = fields;
+    this.configuration = configuration;
+    this.badRecordLogger = badRecordLogger;
+  }
+
+  @Override
+  public void initialize() throws IOException {
+    CacheProvider cacheProvider = CacheProvider.getInstance();
+    cache = cacheProvider.createCache(CacheType.REVERSE_DICTIONARY,
+        configuration.getTableIdentifier().getStorePath());
+    String nullFormat =
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
+            .toString();
+    boolean isEmptyBadRecord = Boolean.parseBoolean(
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD)
+            .toString());
+    List<FieldConverter> fieldConverterList = new ArrayList<>();
+    localCaches = new Map[fields.length];
+    long lruCacheStartTime = System.currentTimeMillis();
+    DictionaryClient client = createDictionaryClient();
+    dictClients.add(client);
+
+    for (int i = 0; i < fields.length; i++) {
+      localCaches[i] = new ConcurrentHashMap<>();
+      FieldConverter fieldConverter = FieldEncoderFactory.getInstance()
+          .createFieldEncoder(fields[i], cache,
+              configuration.getTableIdentifier().getCarbonTableIdentifier(), i, nullFormat, client,
+              configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(),
+              localCaches[i], isEmptyBadRecord);
+      fieldConverterList.add(fieldConverter);
+    }
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+        .recordLruCacheLoadTime((System.currentTimeMillis() - lruCacheStartTime) / 1000.0);
+    fieldConverters = fieldConverterList.toArray(new FieldConverter[fieldConverterList.size()]);
+    logHolder = new BadRecordLogHolder();
+  }
+
+  private DictionaryClient createDictionaryClient() {
+    // for one pass load, start the dictionary client
+    if (configuration.getUseOnePass()) {
+      if (executorService == null) {
+        executorService = Executors.newCachedThreadPool();
+      }
+      Future<DictionaryClient> result = executorService.submit(new Callable<DictionaryClient>() {
+        @Override
+        public DictionaryClient call() throws Exception {
+          Thread.currentThread().setName("Dictionary client");
+          DictionaryClient dictionaryClient = new DictionaryClient();
+          dictionaryClient.startClient(configuration.getDictionaryServerHost(),
+              configuration.getDictionaryServerPort());
+          return dictionaryClient;
+        }
+      });
+
+      try {
+        // wait for client initialization finished, or will raise null pointer exception
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        LOGGER.error(e);
+        throw new RuntimeException(e);
+      }
+
+      try {
+        return result.get();
+      } catch (InterruptedException | ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public CarbonRow convert(CarbonRow row) throws CarbonDataLoadingException {
+    //TODO: only copy if it is bad record
+    CarbonRow copy = row.getCopy();
+    logHolder.setLogged(false);
+    logHolder.clear();
+    for (int i = 0; i < fieldConverters.length; i++) {
+      fieldConverters[i].convert(row, logHolder);
+      if (!logHolder.isLogged() && logHolder.isBadRecordNotAdded()) {
+        badRecordLogger.addBadRecordsToBuilder(copy.getData(), logHolder.getReason());
+        if (badRecordLogger.isDataLoadFail()) {
+          String error = "Data load failed due to bad record: " + logHolder.getReason() +
+              "Please enable bad record logger to know the detail reason.";
+          throw new BadRecordFoundException(error);
+        }
+        logHolder.clear();
+        logHolder.setLogged(true);
+        if (badRecordLogger.isBadRecordConvertNullDisable()) {
+          return null;
+        }
+      }
+    }
+    return row;
+  }
+
+  @Override
+  public void finish() {
+    // close dictionary client when finish write
+    if (configuration.getUseOnePass()) {
+      for (DictionaryClient client : dictClients) {
+        if (client != null) {
+          client.shutDown();
+        }
+      }
+      if (null != logHolder) {
+        logHolder.finish();
+      }
+      if (executorService != null) {
+        executorService.shutdownNow();
+        executorService = null;
+      }
+    }
+  }
+
+  @Override
+  public RowConverter createCopyForNewThread() {
+    RowConverterImpl converter =
+        new RowConverterImpl(this.fields, this.configuration, this.badRecordLogger);
+    List<FieldConverter> fieldConverterList = new ArrayList<>();
+    DictionaryClient client = createDictionaryClient();
+    dictClients.add(client);
+    String nullFormat =
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.SERIALIZATION_NULL_FORMAT)
+            .toString();
+    boolean isEmptyBadRecord = Boolean.parseBoolean(
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD)
+            .toString());
+    for (int i = 0; i < fields.length; i++) {
+      FieldConverter fieldConverter = null;
+      try {
+        fieldConverter = FieldEncoderFactory.getInstance().createFieldEncoder(fields[i], cache,
+            configuration.getTableIdentifier().getCarbonTableIdentifier(), i, nullFormat, client,
+            configuration.getUseOnePass(), configuration.getTableIdentifier().getStorePath(),
+            localCaches[i], isEmptyBadRecord);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      fieldConverterList.add(fieldConverter);
+    }
+    converter.fieldConverters =
+        fieldConverterList.toArray(new FieldConverter[fieldConverterList.size()]);
+    converter.logHolder = new BadRecordLogHolder();
+    return converter;
+  }
+
+  @Override public int[] getCardinality() {
+    List<Integer> dimCardinality = new ArrayList<>();
+    if (fieldConverters != null) {
+      for (int i = 0; i < fieldConverters.length; i++) {
+        if (fieldConverters[i] instanceof AbstractDictionaryFieldConverterImpl) {
+          ((AbstractDictionaryFieldConverterImpl) fieldConverters[i])
+              .fillColumnCardinality(dimCardinality);
+        }
+      }
+    }
+    int[] cardinality = new int[dimCardinality.size()];
+    for (int i = 0; i < dimCardinality.size(); i++) {
+      cardinality[i] = dimCardinality.get(i);
+    }
+    return cardinality;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/BlockDetails.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/BlockDetails.java b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/BlockDetails.java
new file mode 100644
index 0000000..d0c8a73
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/BlockDetails.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.csvinput;
+
+import java.io.Serializable;
+
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+
+/**
+ * blocks info
+ */
+public class BlockDetails extends FileSplit implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 2293906691860002339L;
+  //block offset
+  private long blockOffset;
+  //block length
+  private long blockLength;
+  //file path which block belong to
+  private String filePath;
+  // locations where this block exists
+  private String[] locations;
+
+  public BlockDetails(Path filePath, long blockOffset, long blockLength, String[] locations) {
+    super(filePath, blockOffset, blockLength, locations);
+    this.filePath = filePath.toString();
+    this.blockOffset = blockOffset;
+    this.blockLength = blockLength;
+    this.locations = locations;
+  }
+
+  public long getBlockOffset() {
+    return blockOffset;
+  }
+
+  public long getBlockLength() {
+    return blockLength;
+  }
+
+  public String getFilePath() {
+    return FileFactory.getUpdatedFilePath(filePath);
+  }
+
+  public void setFilePath(String filePath) {
+    this.filePath = filePath;
+  }
+
+  public String[] getLocations() {
+    return locations;
+  }
+
+  /** The file containing this split's data. */
+  @Override
+  public Path getPath() { return new Path(filePath); }
+
+  /** The position of the first byte in the file to process. */
+  @Override
+  public long getStart() { return blockOffset; }
+
+  /** The number of bytes in the file to process. */
+  @Override
+  public long getLength() { return blockLength; }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/BoundedInputStream.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/BoundedInputStream.java b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/BoundedInputStream.java
new file mode 100644
index 0000000..6fe9107
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/csvinput/BoundedInputStream.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.csvinput;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Customarized reader class to read data from file
+ * untill the upper threshold reached.
+ */
+public class BoundedInputStream extends InputStream {
+
+  /**
+   * byte value of the new line character
+   */
+  private static final byte END_OF_LINE_BYTE_VALUE = '\n';
+
+  /**
+   * number of extra character to read
+   */
+  private static final int NUMBER_OF_EXTRA_CHARACTER_TO_READ = 100;
+
+  /**
+   * number of bytes remaining
+   */
+  private long remaining;
+  /**
+   * to check whether end of line is found
+   */
+  private boolean endOfLineFound = false;
+
+  private DataInputStream in;
+
+  public BoundedInputStream(DataInputStream in, long limit) {
+    this.in = in;
+    this.remaining = limit;
+  }
+
+  /**
+   * Below method will be used to read the data from file
+   *
+   * @throws IOException
+   *           problem while reading
+   */
+  @Override
+  public int read() throws IOException {
+    if (this.remaining == 0) {
+      return -1;
+    } else {
+      int var1 = this.in.read();
+      if (var1 >= 0) {
+        --this.remaining;
+      }
+
+      return var1;
+    }
+  }
+
+  /**
+   * Below method will be used to read the data from file. If limit reaches in
+   * that case it will read until new line character is reached
+   *
+   * @param buffer
+   *          buffer in which data will be read
+   * @param offset
+   *          from position to buffer will be filled
+   * @param length
+   *          number of character to be read
+   * @throws IOException
+   *           problem while reading
+   */
+  @Override
+  public int read(byte[] buffer, int offset, int length) throws IOException {
+    if (this.remaining == 0) {
+      return -1;
+    } else {
+      if (this.remaining < length) {
+        length = (int) this.remaining;
+      }
+
+      length = this.in.read(buffer, offset, length);
+      if (length >= 0) {
+        this.remaining -= length;
+        if (this.remaining == 0 && !endOfLineFound) {
+          endOfLineFound = true;
+          this.remaining += NUMBER_OF_EXTRA_CHARACTER_TO_READ;
+        } else if (endOfLineFound) {
+          int end = offset + length;
+          for (int i = offset; i < end; i++) {
+            if (buffer[i] == END_OF_LINE_BYTE_VALUE) {
+              this.remaining = 0;
+              return (i - offset) + 1;
+            }
+          }
+          this.remaining += NUMBER_OF_EXTRA_CHARACTER_TO_READ;
+        }
+      }
+      return length;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (in != null) {
+      in.close();
+    }
+  }
+
+  public long getRemaining() {
+    return  this.remaining;
+  }
+
+}


[18/50] [abbrv] carbondata git commit: [CARBONDATA-1505] Get the detailed blocklet information using default BlockletDataMap for other datamaps

Posted by ja...@apache.org.
[CARBONDATA-1505] Get the detailed blocklet information using default BlockletDataMap for other datamaps

All the detail information of blocklet which is need for exceuting query is present only BlockletDataMap. It is actually default datamap.
So if new datamap is added then it gives only information of blocklet and blockid, it is insuffucient information to exceute query.
Now this PR adds the functionality of retrieving detailed blocklet information from the BlockletDataMap based on block and blocklet id. So now new datamaps can only concentrate on business logic and return only block and blockletid.

This closes #1376


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

Branch: refs/heads/streaming_ingest
Commit: 28f78b2fc1eda650d1e7a7ea48258ba911361b1a
Parents: eb771f5
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Thu Sep 21 11:10:57 2017 +0530
Committer: Jacky Li <ja...@qq.com>
Committed: Thu Sep 28 18:16:17 2017 +0800

----------------------------------------------------------------------
 .../core/datamap/DataMapStoreManager.java       | 22 +++++-
 .../carbondata/core/datamap/TableDataMap.java   | 35 ++++++---
 .../carbondata/core/indexstore/Blocklet.java    | 49 ------------
 .../core/indexstore/BlockletDetailsFetcher.java | 47 +++++++++++
 .../core/indexstore/ExtendedBlocklet.java       | 82 ++++++++++++++++++++
 .../TableBlockIndexUniqueIdentifier.java        |  4 +
 .../blockletindex/BlockletDataMap.java          | 11 ++-
 .../blockletindex/BlockletDataMapFactory.java   | 64 ++++++++++++++-
 .../hadoop/api/CarbonTableInputFormat.java      | 23 ++++--
 .../carbondata/hadoop/api/DataMapJob.java       |  5 +-
 .../hadoop/api/DistributableDataMapFormat.java  | 14 ++--
 .../carbondata/spark/rdd/SparkDataMapJob.scala  | 12 +--
 12 files changed, 279 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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 f19e733..2b5d5cd 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
@@ -25,6 +25,9 @@ import java.util.concurrent.ConcurrentHashMap;
 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.blockletindex.BlockletDataMap;
+import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
 import org.apache.carbondata.core.mutate.UpdateVO;
@@ -109,7 +112,13 @@ public final class DataMapStoreManager {
           (Class<? extends DataMapFactory>) Class.forName(factoryClassName);
       DataMapFactory dataMapFactory = factoryClass.newInstance();
       dataMapFactory.init(identifier, dataMapName);
-      dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory);
+      BlockletDetailsFetcher blockletDetailsFetcher;
+      if (dataMapFactory instanceof BlockletDetailsFetcher) {
+        blockletDetailsFetcher = (BlockletDetailsFetcher) dataMapFactory;
+      } else {
+        blockletDetailsFetcher = getBlockletDetailsFetcher(identifier);
+      }
+      dataMap = new TableDataMap(identifier, dataMapName, dataMapFactory, blockletDetailsFetcher);
     } catch (Exception e) {
       LOGGER.error(e);
       throw new RuntimeException(e);
@@ -152,6 +161,17 @@ public final class DataMapStoreManager {
   }
 
   /**
+   * Get the blocklet datamap factory to get the detail information of blocklets
+   * @param identifier
+   * @return
+   */
+  private BlockletDetailsFetcher getBlockletDetailsFetcher(AbsoluteTableIdentifier identifier) {
+    TableDataMap blockletMap =
+        getDataMap(identifier, BlockletDataMap.NAME, BlockletDataMapFactory.class.getName());
+    return (BlockletDetailsFetcher) blockletMap.getDataMapFactory();
+  }
+
+  /**
    * Returns the singleton instance
    * @return
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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 66bb257..3e5e9e4 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
@@ -25,6 +25,8 @@ import org.apache.carbondata.core.datamap.dev.DataMapFactory;
 import org.apache.carbondata.core.events.ChangeEvent;
 import org.apache.carbondata.core.events.EventListener;
 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.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 
@@ -40,14 +42,17 @@ public final class TableDataMap implements EventListener {
 
   private DataMapFactory dataMapFactory;
 
+  private BlockletDetailsFetcher blockletDetailsFetcher;
+
   /**
    * It is called to initialize and load the required table datamap metadata.
    */
-  public TableDataMap(AbsoluteTableIdentifier identifier,
-      String dataMapName, DataMapFactory dataMapFactory) {
+  public TableDataMap(AbsoluteTableIdentifier identifier, String dataMapName,
+      DataMapFactory dataMapFactory, BlockletDetailsFetcher blockletDetailsFetcher) {
     this.identifier = identifier;
     this.dataMapName = dataMapName;
     this.dataMapFactory = dataMapFactory;
+    this.blockletDetailsFetcher = blockletDetailsFetcher;
   }
 
   /**
@@ -57,21 +62,24 @@ public final class TableDataMap implements EventListener {
    * @param filterExp
    * @return
    */
-  public List<Blocklet> prune(List<String> segmentIds, FilterResolverIntf filterExp)
+  public List<ExtendedBlocklet> prune(List<String> segmentIds, FilterResolverIntf filterExp)
       throws IOException {
-    List<Blocklet> blocklets = new ArrayList<>();
+    List<ExtendedBlocklet> blocklets = new ArrayList<>();
     for (String segmentId : segmentIds) {
+      List<Blocklet> pruneBlocklets = new ArrayList<>();
       List<DataMap> dataMaps = dataMapFactory.getDataMaps(segmentId);
       for (DataMap dataMap : dataMaps) {
-        List<Blocklet> pruneBlocklets = dataMap.prune(filterExp);
-        blocklets.addAll(addSegmentId(pruneBlocklets, segmentId));
+        pruneBlocklets.addAll(dataMap.prune(filterExp));
       }
+      blocklets.addAll(addSegmentId(blockletDetailsFetcher
+          .getExtendedBlocklets(pruneBlocklets, segmentId), segmentId));
     }
     return blocklets;
   }
 
-  private List<Blocklet> addSegmentId(List<Blocklet> pruneBlocklets, String segmentId) {
-    for (Blocklet blocklet : pruneBlocklets) {
+  private List<ExtendedBlocklet> addSegmentId(List<ExtendedBlocklet> pruneBlocklets,
+      String segmentId) {
+    for (ExtendedBlocklet blocklet : pruneBlocklets) {
       blocklet.setSegmentId(segmentId);
     }
     return pruneBlocklets;
@@ -107,12 +115,17 @@ public final class TableDataMap implements EventListener {
    * @param filterExp
    * @return
    */
-  public List<Blocklet> prune(DataMapDistributable distributable, FilterResolverIntf filterExp) {
+  public List<ExtendedBlocklet> prune(DataMapDistributable distributable,
+      FilterResolverIntf filterExp) throws IOException {
+    List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
     List<Blocklet> blocklets = dataMapFactory.getDataMap(distributable).prune(filterExp);
     for (Blocklet blocklet: blocklets) {
-      blocklet.setSegmentId(distributable.getSegmentId());
+      ExtendedBlocklet detailedBlocklet =
+          blockletDetailsFetcher.getExtendedBlocklet(blocklet, distributable.getSegmentId());
+      detailedBlocklet.setSegmentId(distributable.getSegmentId());
+      detailedBlocklets.add(detailedBlocklet);
     }
-    return blocklets;
+    return detailedBlocklets;
   }
 
   @Override public void fireEvent(ChangeEvent event) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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 919a48d..d84f3f6 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,16 +16,8 @@
  */
 package org.apache.carbondata.core.indexstore;
 
-import java.io.IOException;
 import java.io.Serializable;
 
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-
 /**
  * Blocklet
  */
@@ -33,16 +25,8 @@ public class Blocklet implements Serializable {
 
   private String path;
 
-  private String segmentId;
-
   private String blockletId;
 
-  private BlockletDetailInfo detailInfo;
-
-  private long length;
-
-  private String[] location;
-
   public Blocklet(String path, String blockletId) {
     this.path = path;
     this.blockletId = blockletId;
@@ -56,37 +40,4 @@ public class Blocklet implements Serializable {
     return blockletId;
   }
 
-  public BlockletDetailInfo getDetailInfo() {
-    return detailInfo;
-  }
-
-  public void setDetailInfo(BlockletDetailInfo detailInfo) {
-    this.detailInfo = detailInfo;
-  }
-
-  public void updateLocations() throws IOException {
-    Path path = new Path(this.path);
-    FileSystem fs = path.getFileSystem(FileFactory.getConfiguration());
-    RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(path);
-    LocatedFileStatus fileStatus = iter.next();
-    location = fileStatus.getBlockLocations()[0].getHosts();
-    length = fileStatus.getLen();
-  }
-
-  public String[] getLocations() throws IOException {
-    return location;
-  }
-
-  public long getLength() throws IOException {
-    return length;
-  }
-
-  public String getSegmentId() {
-    return segmentId;
-  }
-
-  public void setSegmentId(String segmentId) {
-    this.segmentId = segmentId;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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
new file mode 100644
index 0000000..21ecba1
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/BlockletDetailsFetcher.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.List;
+
+/**
+ * Fetches the detailed blocklet which has more information to execute the query
+ */
+public interface BlockletDetailsFetcher {
+
+  /**
+   * Get the blocklet detail information based on blockletid, blockid and segmentid.
+   *
+   * @param blocklets
+   * @param segmentId
+   * @return
+   * @throws IOException
+   */
+  List<ExtendedBlocklet> getExtendedBlocklets(List<Blocklet> blocklets, String segmentId)
+      throws IOException;
+
+  /**
+   * Get the blocklet detail information based on blockletid, blockid and segmentid.
+   *
+   * @param blocklet
+   * @param segmentId
+   * @return
+   * @throws IOException
+   */
+  ExtendedBlocklet getExtendedBlocklet(Blocklet blocklet, String segmentId) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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
new file mode 100644
index 0000000..e0cfefb
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.impl.FileFactory;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+
+/**
+ * Detailed blocklet information
+ */
+public class ExtendedBlocklet extends Blocklet {
+
+  private String segmentId;
+
+  private BlockletDetailInfo detailInfo;
+
+  private long length;
+
+  private String[] location;
+
+  public ExtendedBlocklet(String path, String blockletId) {
+    super(path, blockletId);
+  }
+
+  public BlockletDetailInfo getDetailInfo() {
+    return detailInfo;
+  }
+
+  public void setDetailInfo(BlockletDetailInfo detailInfo) {
+    this.detailInfo = detailInfo;
+  }
+
+  /**
+   * It gets the hdfs block locations and length for this blocklet. It is used internally to get the
+   * locations for allocating tasks.
+   * @throws IOException
+   */
+  public void updateLocations() throws IOException {
+    Path path = new Path(getPath());
+    FileSystem fs = path.getFileSystem(FileFactory.getConfiguration());
+    RemoteIterator<LocatedFileStatus> iter = fs.listLocatedStatus(path);
+    LocatedFileStatus fileStatus = iter.next();
+    location = fileStatus.getBlockLocations()[0].getHosts();
+    length = fileStatus.getLen();
+  }
+
+  public String[] getLocations() throws IOException {
+    return location;
+  }
+
+  public long getLength() throws IOException {
+    return length;
+  }
+
+  public String getSegmentId() {
+    return segmentId;
+  }
+
+  public void setSegmentId(String segmentId) {
+    this.segmentId = segmentId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/core/src/main/java/org/apache/carbondata/core/indexstore/TableBlockIndexUniqueIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/indexstore/TableBlockIndexUniqueIdentifier.java b/core/src/main/java/org/apache/carbondata/core/indexstore/TableBlockIndexUniqueIdentifier.java
index 7e2bc0e..18357ac 100644
--- a/core/src/main/java/org/apache/carbondata/core/indexstore/TableBlockIndexUniqueIdentifier.java
+++ b/core/src/main/java/org/apache/carbondata/core/indexstore/TableBlockIndexUniqueIdentifier.java
@@ -100,4 +100,8 @@ public class TableBlockIndexUniqueIdentifier {
     result = 31 * result + carbonIndexFileName.hashCode();
     return result;
   }
+
+  public String getCarbonIndexFileName() {
+    return carbonIndexFileName;
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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 57211fd..adcb1a6 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
@@ -39,6 +39,7 @@ import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.indexstore.Blocklet;
 import org.apache.carbondata.core.indexstore.BlockletDetailInfo;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.UnsafeMemoryDMStore;
 import org.apache.carbondata.core.indexstore.row.DataMapRow;
 import org.apache.carbondata.core.indexstore.row.DataMapRowImpl;
@@ -286,6 +287,12 @@ public class BlockletDataMap implements DataMap, Cacheable {
     return blocklets;
   }
 
+  public ExtendedBlocklet getDetailedBlocklet(String blockletId) {
+    int index = Integer.parseInt(blockletId);
+    DataMapRow unsafeRow = unsafeMemoryDMStore.getUnsafeRow(index);
+    return createBlocklet(unsafeRow, index);
+  }
+
   private byte[][] getMinMaxValue(DataMapRow row, int index) {
     DataMapRow minMaxRow = row.getRow(index);
     byte[][] minMax = new byte[minMaxRow.getColumnCount()][];
@@ -295,8 +302,8 @@ public class BlockletDataMap implements DataMap, Cacheable {
     return minMax;
   }
 
-  private Blocklet createBlocklet(DataMapRow row, int blockletId) {
-    Blocklet blocklet = new Blocklet(
+  private ExtendedBlocklet createBlocklet(DataMapRow row, int blockletId) {
+    ExtendedBlocklet blocklet = new ExtendedBlocklet(
         new String(row.getByteArray(FILE_PATH_INDEX), CarbonCommonConstants.DEFAULT_CHARSET_CLASS),
         blockletId + "");
     BlockletDetailInfo detailInfo = new BlockletDetailInfo();

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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 d734d81..5edc5b7 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
@@ -34,8 +34,12 @@ 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.events.ChangeEvent;
+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.TableBlockIndexUniqueIdentifier;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -45,7 +49,7 @@ import org.apache.hadoop.fs.RemoteIterator;
 /**
  * Table map for blocklet
  */
-public class BlockletDataMapFactory implements DataMapFactory {
+public class BlockletDataMapFactory implements DataMapFactory, BlockletDetailsFetcher {
 
   private AbsoluteTableIdentifier identifier;
 
@@ -69,6 +73,13 @@ public class BlockletDataMapFactory implements DataMapFactory {
   @Override
   public List<DataMap> getDataMaps(String segmentId) throws IOException {
     List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
+        getTableBlockIndexUniqueIdentifiers(segmentId);
+    return cache.getAll(tableBlockIndexUniqueIdentifiers);
+  }
+
+  private List<TableBlockIndexUniqueIdentifier> getTableBlockIndexUniqueIdentifiers(
+      String segmentId) {
+    List<TableBlockIndexUniqueIdentifier> tableBlockIndexUniqueIdentifiers =
         segmentMap.get(segmentId);
     if (tableBlockIndexUniqueIdentifiers == null) {
       tableBlockIndexUniqueIdentifiers = new ArrayList<>();
@@ -79,8 +90,55 @@ public class BlockletDataMapFactory implements DataMapFactory {
       }
       segmentMap.put(segmentId, tableBlockIndexUniqueIdentifiers);
     }
+    return tableBlockIndexUniqueIdentifiers;
+  }
 
-    return cache.getAll(tableBlockIndexUniqueIdentifiers);
+  /**
+   * Get the blocklet detail information based on blockletid, blockid and segmentid. This method is
+   * exclusively for BlockletDataMapFactory as detail information is only available in this default
+   * datamap.
+   */
+  @Override
+  public List<ExtendedBlocklet> getExtendedBlocklets(List<Blocklet> blocklets, String segmentId)
+      throws IOException {
+    List<ExtendedBlocklet> detailedBlocklets = new ArrayList<>();
+    // If it is already detailed blocklet then type cast and return same
+    if (blocklets.size() > 0 && blocklets.get(0) instanceof ExtendedBlocklet) {
+      for (Blocklet blocklet : blocklets) {
+        detailedBlocklets.add((ExtendedBlocklet) blocklet);
+      }
+      return detailedBlocklets;
+    }
+    List<TableBlockIndexUniqueIdentifier> identifiers =
+        getTableBlockIndexUniqueIdentifiers(segmentId);
+    // Retrieve each blocklets detail information from blocklet datamap
+    for (Blocklet blocklet : blocklets) {
+      detailedBlocklets.add(getExtendedBlocklet(identifiers, blocklet));
+    }
+    return detailedBlocklets;
+  }
+
+  @Override
+  public ExtendedBlocklet getExtendedBlocklet(Blocklet blocklet, String segmentId)
+      throws IOException {
+    if (blocklet instanceof ExtendedBlocklet) {
+      return (ExtendedBlocklet) blocklet;
+    }
+    List<TableBlockIndexUniqueIdentifier> identifiers =
+        getTableBlockIndexUniqueIdentifiers(segmentId);
+    return getExtendedBlocklet(identifiers, blocklet);
+  }
+
+  private ExtendedBlocklet getExtendedBlocklet(List<TableBlockIndexUniqueIdentifier> identifiers,
+      Blocklet blocklet) throws IOException {
+    String carbonIndexFileName = CarbonTablePath.getCarbonIndexFileName(blocklet.getPath());
+    for (TableBlockIndexUniqueIdentifier identifier : identifiers) {
+      if (identifier.getCarbonIndexFileName().equals(carbonIndexFileName)) {
+        DataMap dataMap = cache.get(identifier);
+        return ((BlockletDataMap) dataMap).getDetailedBlocklet(blocklet.getBlockletId());
+      }
+    }
+    throw new IOException("Blocklet with blockid " + blocklet.getPath() + " not found ");
   }
 
   private CarbonFile[] getCarbonIndexFiles(String segmentId) {
@@ -131,7 +189,7 @@ public class BlockletDataMapFactory implements DataMapFactory {
 
   @Override
   public void clear() {
-    for (String segmentId: segmentMap.keySet().toArray(new String[segmentMap.size()])) {
+    for (String segmentId : segmentMap.keySet().toArray(new String[segmentMap.size()])) {
       clear(segmentId);
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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 55d22d1..314ffd5 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
@@ -21,12 +21,18 @@ import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.lang.reflect.Constructor;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMap;
 import org.apache.carbondata.core.indexstore.blockletindex.BlockletDataMapFactory;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
@@ -533,7 +539,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
         .getDataMap(absoluteTableIdentifier, BlockletDataMap.NAME,
             BlockletDataMapFactory.class.getName());
     DataMapJob dataMapJob = getDataMapJob(job.getConfiguration());
-    List<Blocklet> prunedBlocklets;
+    List<ExtendedBlocklet> prunedBlocklets;
     if (dataMapJob != null) {
       DistributableDataMapFormat datamapDstr =
           new DistributableDataMapFormat(absoluteTableIdentifier, BlockletDataMap.NAME,
@@ -549,7 +555,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     if (partitionInfo != null) {
       partitionIdList = partitionInfo.getPartitionIds();
     }
-    for (Blocklet blocklet : prunedBlocklets) {
+    for (ExtendedBlocklet blocklet : prunedBlocklets) {
       int partitionId = CarbonTablePath.DataFileUtil.getTaskIdFromTaskNo(
           CarbonTablePath.DataFileUtil.getTaskNo(blocklet.getPath().toString()));
 
@@ -580,7 +586,7 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
     return resultFilterredBlocks;
   }
 
-  private org.apache.carbondata.hadoop.CarbonInputSplit convertToCarbonInputSplit(Blocklet blocklet)
+  private CarbonInputSplit convertToCarbonInputSplit(ExtendedBlocklet blocklet)
       throws IOException {
     blocklet.updateLocations();
     org.apache.carbondata.hadoop.CarbonInputSplit split =
@@ -715,9 +721,10 @@ public class CarbonTableInputFormat<T> extends FileInputFormat<Void, T> {
         new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     Map<String, Long> segmentAndBlockCountMapping =
         new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<Blocklet> blocklets = blockletMap.prune(validAndInvalidSegments.getValidSegments(), null);
-    for (Blocklet blocklet : blocklets) {
-      String blockName = blocklet.getPath().toString();
+    List<ExtendedBlocklet> blocklets =
+        blockletMap.prune(validAndInvalidSegments.getValidSegments(), null);
+    for (ExtendedBlocklet blocklet : blocklets) {
+      String blockName = blocklet.getPath();
       blockName = CarbonTablePath.getCarbonDataFileName(blockName);
       blockName = blockName + CarbonTablePath.getCarbonDataExtension();
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java
index e33c356..fad2336 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DataMapJob.java
@@ -19,7 +19,7 @@ package org.apache.carbondata.hadoop.api;
 import java.io.Serializable;
 import java.util.List;
 
-import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 
 /**
@@ -28,6 +28,7 @@ import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
  */
 public interface DataMapJob extends Serializable {
 
-  List<Blocklet> execute(DistributableDataMapFormat dataMapFormat, FilterResolverIntf resolverIntf);
+  List<ExtendedBlocklet> execute(DistributableDataMapFormat dataMapFormat,
+      FilterResolverIntf resolverIntf);
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
index 653e33f..66a06ba 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/DistributableDataMapFormat.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.carbondata.core.datamap.DataMapDistributable;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.datamap.TableDataMap;
-import org.apache.carbondata.core.indexstore.Blocklet;
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
 import org.apache.carbondata.hadoop.util.ObjectSerializationUtil;
@@ -40,7 +40,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 /**
  * Input format for datamaps, it makes the datamap pruning distributable.
  */
-public class DistributableDataMapFormat extends FileInputFormat<Void, Blocklet> implements
+public class DistributableDataMapFormat extends FileInputFormat<Void, ExtendedBlocklet> implements
     Serializable {
 
   private static final String FILTER_EXP = "mapreduce.input.distributed.datamap.filter";
@@ -89,11 +89,11 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, Blocklet>
   }
 
   @Override
-  public RecordReader<Void, Blocklet> createRecordReader(InputSplit inputSplit,
+  public RecordReader<Void, ExtendedBlocklet> createRecordReader(InputSplit inputSplit,
       TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException {
-    return new RecordReader<Void, Blocklet>() {
-      private Iterator<Blocklet> blockletIterator;
-      private Blocklet currBlocklet;
+    return new RecordReader<Void, ExtendedBlocklet>() {
+      private Iterator<ExtendedBlocklet> blockletIterator;
+      private ExtendedBlocklet currBlocklet;
 
       @Override
       public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
@@ -124,7 +124,7 @@ public class DistributableDataMapFormat extends FileInputFormat<Void, Blocklet>
       }
 
       @Override
-      public Blocklet getCurrentValue() throws IOException, InterruptedException {
+      public ExtendedBlocklet getCurrentValue() throws IOException, InterruptedException {
         return currBlocklet;
       }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/28f78b2f/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 f9e4f8d..fbe9377 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
@@ -29,7 +29,7 @@ import org.apache.spark.{Partition, SparkContext, TaskContext, TaskKilledExcepti
 
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.indexstore.Blocklet
+import org.apache.carbondata.core.indexstore.ExtendedBlocklet
 import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf
 import org.apache.carbondata.hadoop.api.{DataMapJob, DistributableDataMapFormat}
 
@@ -39,7 +39,7 @@ import org.apache.carbondata.hadoop.api.{DataMapJob, DistributableDataMapFormat}
 class SparkDataMapJob extends DataMapJob {
 
   override def execute(dataMapFormat: DistributableDataMapFormat,
-      resolverIntf: FilterResolverIntf): util.List[Blocklet] = {
+      resolverIntf: FilterResolverIntf): util.List[ExtendedBlocklet] = {
     new DataMapPruneRDD(SparkContext.getOrCreate(), dataMapFormat, resolverIntf).collect().toList
       .asJava
   }
@@ -60,7 +60,7 @@ class DataMapRDDPartition(rddId: Int, idx: Int, val inputSplit: InputSplit) exte
 class DataMapPruneRDD(sc: SparkContext,
     dataMapFormat: DistributableDataMapFormat,
     resolverIntf: FilterResolverIntf)
-  extends CarbonRDD[(Blocklet)](sc, Nil) {
+  extends CarbonRDD[(ExtendedBlocklet)](sc, Nil) {
 
   private val jobTrackerId: String = {
     val formatter = new SimpleDateFormat("yyyyMMddHHmm")
@@ -68,7 +68,7 @@ class DataMapPruneRDD(sc: SparkContext,
   }
 
   override def internalCompute(split: Partition,
-      context: TaskContext): Iterator[Blocklet] = {
+      context: TaskContext): Iterator[ExtendedBlocklet] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
     val status = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
     val attemptId = new TaskAttemptID(jobTrackerId, id, TaskType.MAP, split.index, 0)
@@ -77,7 +77,7 @@ class DataMapPruneRDD(sc: SparkContext,
     DistributableDataMapFormat.setFilterExp(attemptContext.getConfiguration, resolverIntf)
     val reader = dataMapFormat.createRecordReader(inputSplit, attemptContext)
     reader.initialize(inputSplit, attemptContext)
-    val iter = new Iterator[Blocklet] {
+    val iter = new Iterator[ExtendedBlocklet] {
 
       private var havePair = false
       private var finished = false
@@ -93,7 +93,7 @@ class DataMapPruneRDD(sc: SparkContext,
         !finished
       }
 
-      override def next(): Blocklet = {
+      override def next(): ExtendedBlocklet = {
         if (!hasNext) {
           throw new java.util.NoSuchElementException("End of stream")
         }


[08/50] [abbrv] carbondata git commit: [CARBONDATA-1151] Refactor all carbon command to separate file in spark2 integration

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
new file mode 100644
index 0000000..21b974a
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/AlterTableSplitCarbonPartitionCommand.scala
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.partition
+
+import java.text.SimpleDateFormat
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableSplitPartitionModel, DataProcessCommand, RunnableCommand, SchemaProcessCommand}
+import org.apache.spark.sql.hive.{CarbonMetaStore, CarbonRelation}
+import org.apache.spark.util.{AlterTableUtil, PartitionUtils}
+
+import org.apache.carbondata.common.logging.{LogService, LogServiceFactory}
+import org.apache.carbondata.core.cache.CacheProvider
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{CarbonMetadata, CarbonTableIdentifier}
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.PartitionInfo
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.mutate.CarbonUpdateUtil
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.processing.model.{CarbonDataLoadSchema, CarbonLoadModel}
+import org.apache.carbondata.spark.rdd.CarbonDataRDDFactory
+
+/**
+ * Command for Alter Table Add & Split partition
+ * Add is a special case of Splitting the default partition (part0)
+ */
+case class AlterTableSplitCarbonPartitionCommand(
+    splitPartitionModel: AlterTableSplitPartitionModel)
+  extends RunnableCommand with DataProcessCommand with SchemaProcessCommand {
+
+  val oldPartitionIds: util.ArrayList[Int] = new util.ArrayList[Int]()
+
+  // TODO will add rollback function in case of process data failure
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+    processData(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+    val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
+    val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val tableName = splitPartitionModel.tableName
+    val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
+      .asInstanceOf[CarbonRelation]
+    val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+    val storePath = relation.tableMeta.storePath
+    if (relation == null) {
+      sys.error(s"Table $dbName.$tableName does not exist")
+    }
+    carbonMetaStore.checkSchemasModifiedTimeAndReloadTables(storePath)
+    if (null == CarbonMetadata.getInstance.getCarbonTable(dbName + "_" + tableName)) {
+      LOGGER.error(s"Alter table failed. table not found: $dbName.$tableName")
+      sys.error(s"Alter table failed. table not found: $dbName.$tableName")
+    }
+    val table = relation.tableMeta.carbonTable
+    val partitionInfo = table.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
+    oldPartitionIds.addAll(partitionIds.asJava)
+
+    if (partitionInfo == null) {
+      sys.error(s"Table $tableName is not a partition table.")
+    }
+    if (partitionInfo.getPartitionType == PartitionType.HASH) {
+      sys.error(s"Hash partition table cannot be added or split!")
+    }
+
+    updatePartitionInfo(partitionInfo, partitionIds)
+
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
+    val schemaFilePath = carbonTablePath.getSchemaFilePath
+    // read TableInfo
+    val tableInfo = carbonMetaStore.getThriftTableInfo(carbonTablePath)(sparkSession)
+    val schemaConverter = new ThriftWrapperSchemaConverterImpl()
+    val wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(tableInfo,
+      dbName, tableName, storePath)
+    val tableSchema = wrapperTableInfo.getFactTable
+    tableSchema.setPartitionInfo(partitionInfo)
+    wrapperTableInfo.setFactTable(tableSchema)
+    wrapperTableInfo.setLastUpdatedTime(System.currentTimeMillis())
+    val thriftTable =
+      schemaConverter.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+    carbonMetaStore.updateMetadataByThriftTable(schemaFilePath, thriftTable,
+      dbName, tableName, storePath)
+    CarbonUtil.writeThriftTableToSchemaFile(schemaFilePath, thriftTable)
+    // update the schema modified time
+    carbonMetaStore.updateAndTouchSchemasUpdatedTime(storePath)
+    sparkSession.catalog.refreshTable(tableName)
+    Seq.empty
+  }
+
+  private def updatePartitionInfo(partitionInfo: PartitionInfo,
+      partitionIds: List[Int]) = {
+    val dateFormatter = new SimpleDateFormat(CarbonProperties.getInstance
+      .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
+
+    val timestampFormatter = new SimpleDateFormat(CarbonProperties.getInstance
+      .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
+
+    PartitionUtils.updatePartitionInfo(
+      partitionInfo,
+      partitionIds,
+      splitPartitionModel.partitionId.toInt,
+      splitPartitionModel.splitInfo,
+      timestampFormatter,
+      dateFormatter)
+  }
+
+  override def processData(sparkSession: SparkSession): Seq[Row] = {
+    val dbName = splitPartitionModel.databaseName.getOrElse(sparkSession.catalog.currentDatabase)
+    val tableName = splitPartitionModel.tableName
+    var locks = List.empty[ICarbonLock]
+    var success = false
+    try {
+      val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+        LockUsage.COMPACTION_LOCK,
+        LockUsage.DELETE_SEGMENT_LOCK,
+        LockUsage.DROP_TABLE_LOCK,
+        LockUsage.CLEAN_FILES_LOCK,
+        LockUsage.ALTER_PARTITION_LOCK)
+      locks = AlterTableUtil.validateTableAndAcquireLock(dbName, tableName,
+        locksToBeAcquired)(sparkSession)
+      val carbonLoadModel = new CarbonLoadModel()
+      val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      val relation = carbonMetaStore.lookupRelation(Option(dbName), tableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+      val storePath = relation.tableMeta.storePath
+      val table = relation.tableMeta.carbonTable
+      val carbonTableIdentifier = relation.tableMeta.carbonTableIdentifier
+      val dataLoadSchema = new CarbonDataLoadSchema(table)
+      carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
+      carbonLoadModel.setTableName(carbonTableIdentifier.getTableName)
+      carbonLoadModel.setDatabaseName(carbonTableIdentifier.getDatabaseName)
+      carbonLoadModel.setStorePath(storePath)
+      val loadStartTime = CarbonUpdateUtil.readCurrentTime
+      carbonLoadModel.setFactTimeStamp(loadStartTime)
+      CarbonDataRDDFactory.alterTableSplitPartition(
+        sparkSession.sqlContext,
+        splitPartitionModel.partitionId.toInt.toString,
+        carbonLoadModel,
+        oldPartitionIds.asScala.toList
+      )
+      success = true
+    } catch {
+      case e: Exception =>
+        success = false
+        sys.error(s"Add/Split Partition failed. Please check logs for more info. ${ e.getMessage }")
+    } finally {
+      AlterTableUtil.releaseLocks(locks)
+      CacheProvider.getInstance().dropAllCache()
+      val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getName)
+      LOGGER.info("Locks released after alter table add/split partition action.")
+      LOGGER.audit("Locks released after alter table add/split partition action.")
+      if (success) {
+        LOGGER.info(s"Alter table add/split partition is successful for table $dbName.$tableName")
+        LOGGER.audit(s"Alter table add/split partition is successful for table $dbName.$tableName")
+      }
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/ShowCarbonPartitionsCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/ShowCarbonPartitionsCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/ShowCarbonPartitionsCommand.scala
new file mode 100644
index 0000000..224304a
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/ShowCarbonPartitionsCommand.scala
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.partition
+
+import org.apache.spark.sql.{AnalysisException, CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.execution.command.{RunnableCommand, SchemaProcessCommand}
+import org.apache.spark.sql.hive.CarbonRelation
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.spark.util.CommonUtil
+
+/**
+ * Command for show table partitions Command
+ */
+private[sql] case class ShowCarbonPartitionsCommand(
+    tableIdentifier: TableIdentifier)
+  extends RunnableCommand with SchemaProcessCommand {
+
+  override val output: Seq[Attribute] = CommonUtil.partitionInfoOutput
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    processSchema(sparkSession)
+  }
+
+  override def processSchema(sparkSession: SparkSession): Seq[Row] = {
+    val relation = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      .lookupRelation(tableIdentifier)(sparkSession).
+      asInstanceOf[CarbonRelation]
+    val carbonTable = relation.tableMeta.carbonTable
+    val tableName = carbonTable.getFactTableName
+    val partitionInfo = carbonTable.getPartitionInfo(
+      carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+    if (partitionInfo == null) {
+      throw new AnalysisException(
+        s"SHOW PARTITIONS is not allowed on a table that is not partitioned: $tableName")
+    }
+    val partitionType = partitionInfo.getPartitionType
+    val columnName = partitionInfo.getColumnSchemaList.get(0).getColumnName
+    val LOGGER = LogServiceFactory.getLogService(ShowCarbonPartitionsCommand.getClass.getName)
+    LOGGER.info("partition column name:" + columnName)
+    CommonUtil.getPartitionInfo(columnName, partitionType, partitionInfo)
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
new file mode 100644
index 0000000..6e6a4b1
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableAddColumnCommand.scala
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.schema
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableColumnSchemaGenerator, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+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.converter.ThriftWrapperSchemaConverterImpl
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.TableInfo
+import org.apache.carbondata.spark.rdd.{AlterTableAddColumnRDD, AlterTableDropColumnRDD}
+
+private[sql] case class AlterTableAddColumnCommand(
+    alterTableAddColumnsModel: AlterTableAddColumnsModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableAddColumnsModel.tableName
+    val dbName = alterTableAddColumnsModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table add columns request has been received for $dbName.$tableName")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    var newCols = Seq[org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema]()
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      // Consider a concurrent scenario where 2 alter operations are executed in parallel. 1st
+      // operation is success and updates the schema file. 2nd operation will get the lock after
+      // completion of 1st operation but as look up relation is called before it will have the
+      // older carbon table and this can lead to inconsistent state in the system. Therefor look
+      // up relation should be called after acquiring the lock
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      // get the latest carbon table and check for column existence
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val thriftTableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val schemaConverter = new ThriftWrapperSchemaConverterImpl()
+      val wrapperTableInfo = schemaConverter
+        .fromExternalToWrapperTableInfo(thriftTableInfo,
+          dbName,
+          tableName,
+          carbonTable.getStorePath)
+      newCols = new AlterTableColumnSchemaGenerator(alterTableAddColumnsModel,
+        dbName,
+        wrapperTableInfo,
+        carbonTablePath,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath, sparkSession.sparkContext).process
+      // generate dictionary files for the newly added columns
+      new AlterTableAddColumnRDD(sparkSession.sparkContext,
+        newCols,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath).collect()
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry
+      schemaEvolutionEntry.setTimeStamp(timeStamp)
+      schemaEvolutionEntry.setAdded(newCols.toList.asJava)
+      val thriftTable = schemaConverter
+        .fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaConverter.fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry),
+          thriftTable)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      LOGGER.info(s"Alter table for add columns is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for add columns is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, "Alter table add columns failed")
+        if (newCols.nonEmpty) {
+          LOGGER.info("Cleaning up the dictionary files as alter table add operation failed")
+          new AlterTableDropColumnRDD(sparkSession.sparkContext,
+            newCols,
+            carbonTable.getCarbonTableIdentifier,
+            carbonTable.getStorePath).collect()
+          AlterTableUtil.revertAddColumnChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table add operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
new file mode 100644
index 0000000..be87bbb
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDataTypeChangeCommand.scala
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.schema
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableDataTypeChangeModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+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.format.{ColumnSchema, SchemaEvolutionEntry, TableInfo}
+import org.apache.carbondata.spark.util.{CarbonScalaUtil, DataTypeConverterUtil}
+
+private[sql] case class AlterTableDataTypeChangeCommand(
+    alterTableDataTypeChangeModel: AlterTableDataTypeChangeModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableDataTypeChangeModel.tableName
+    val dbName = alterTableDataTypeChangeModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table change data type request has been received for $dbName.$tableName")
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    var locks = List.empty[ICarbonLock]
+    // get the latest carbon table and check for column existence
+    var carbonTable: CarbonTable = null
+    var timeStamp = 0L
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      val columnName = alterTableDataTypeChangeModel.columnName
+      val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala.filter(!_.isInvisible)
+      if (!carbonColumns.exists(_.getColName.equalsIgnoreCase(columnName))) {
+        LOGGER.audit(s"Alter table change data type request has failed. " +
+                     s"Column $columnName does not exist")
+        sys.error(s"Column does not exist: $columnName")
+      }
+      val carbonColumn = carbonColumns.filter(_.getColName.equalsIgnoreCase(columnName))
+      if (carbonColumn.size == 1) {
+        CarbonScalaUtil
+          .validateColumnDataType(alterTableDataTypeChangeModel.dataTypeInfo, carbonColumn.head)
+      } else {
+        LOGGER.audit(s"Alter table change data type request has failed. " +
+                     s"Column $columnName is invalid")
+        sys.error(s"Invalid Column: $columnName")
+      }
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val tableInfo: TableInfo = metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      // maintain the added column for schema evolution history
+      var addColumnSchema: ColumnSchema = null
+      var deletedColumnSchema: 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.setPrecision(alterTableDataTypeChangeModel.dataTypeInfo.precision)
+          columnSchema.setScale(alterTableDataTypeChangeModel.dataTypeInfo.scale)
+          addColumnSchema = columnSchema
+        }
+      }
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
+      schemaEvolutionEntry.setAdded(List(addColumnSchema).asJava)
+      schemaEvolutionEntry.setRemoved(List(deletedColumnSchema).asJava)
+      tableInfo.getFact_table.getSchema_evolution.getSchema_evolution_history.get(0)
+        .setTime_stamp(System.currentTimeMillis)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaEvolutionEntry,
+          tableInfo)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      LOGGER.info(s"Alter table for data type change is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for data type change is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception => LOGGER
+        .error("Alter table change datatype failed : " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil.revertDataTypeChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table data type change operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
new file mode 100644
index 0000000..2f1e3d9
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableDropColumnCommand.scala
@@ -0,0 +1,148 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.schema
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.execution.command.{AlterTableDropColumnModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+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.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.path.CarbonStorePath
+import org.apache.carbondata.format.SchemaEvolutionEntry
+import org.apache.carbondata.spark.rdd.AlterTableDropColumnRDD
+
+private[sql] case class AlterTableDropColumnCommand(
+    alterTableDropColumnModel: AlterTableDropColumnModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val tableName = alterTableDropColumnModel.tableName
+    val dbName = alterTableDropColumnModel.databaseName
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    LOGGER.audit(s"Alter table drop columns request has been received for $dbName.$tableName")
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK, LockUsage.COMPACTION_LOCK)
+    // get the latest carbon table and check for column existence
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(dbName, tableName, locksToBeAcquired)(sparkSession)
+      val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+      carbonTable = metastore
+        .lookupRelation(Some(dbName), tableName)(sparkSession).asInstanceOf[CarbonRelation]
+        .tableMeta.carbonTable
+      val partitionInfo = carbonTable.getPartitionInfo(tableName)
+      if (partitionInfo != null) {
+        val partitionColumnSchemaList = partitionInfo.getColumnSchemaList.asScala
+          .map(_.getColumnName)
+        // check each column existence in the table
+        val partitionColumns = alterTableDropColumnModel.columns.filter {
+          tableColumn => partitionColumnSchemaList.contains(tableColumn)
+        }
+        if (partitionColumns.nonEmpty) {
+          throw new UnsupportedOperationException("Partition columns cannot be dropped: " +
+                                                  s"$partitionColumns")
+        }
+      }
+      val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
+      var dictionaryColumns = Seq[org.apache.carbondata.core.metadata.schema.table.column
+      .ColumnSchema]()
+      var keyColumnCountToBeDeleted = 0
+      // TODO: if deleted column list includes bucketted column throw an error
+      alterTableDropColumnModel.columns.foreach { column =>
+        var columnExist = false
+        tableColumns.foreach { tableColumn =>
+          // column should not be already deleted and should exist in the table
+          if (!tableColumn.isInvisible && column.equalsIgnoreCase(tableColumn.getColName)) {
+            if (tableColumn.isDimension) {
+              keyColumnCountToBeDeleted += 1
+              if (tableColumn.hasEncoding(Encoding.DICTIONARY)) {
+                dictionaryColumns ++= Seq(tableColumn.getColumnSchema)
+              }
+            }
+            columnExist = true
+          }
+        }
+        if (!columnExist) {
+          sys.error(s"Column $column does not exists in the table $dbName.$tableName")
+        }
+      }
+      // take the total key column count. key column to be deleted should not
+      // be >= key columns in schema
+      val totalKeyColumnInSchema = tableColumns.count {
+        tableColumn => !tableColumn.isInvisible && tableColumn.isDimension
+      }
+      if (keyColumnCountToBeDeleted >= totalKeyColumnInSchema) {
+        sys.error(s"Alter drop operation failed. AtLeast one key column should exist after drop.")
+      }
+      // read the latest schema file
+      val carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getStorePath,
+        carbonTable.getCarbonTableIdentifier)
+      val tableInfo: org.apache.carbondata.format.TableInfo =
+        metastore.getThriftTableInfo(carbonTablePath)(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
+      alterTableDropColumnModel.columns.foreach { column =>
+        columnSchemaList.foreach { columnSchema =>
+          if (!columnSchema.invisible && column.equalsIgnoreCase(columnSchema.column_name)) {
+            deletedColumnSchema += columnSchema.deepCopy
+            columnSchema.invisible = true
+          }
+        }
+      }
+      // add deleted columns to schema evolution history and update the schema
+      timeStamp = System.currentTimeMillis
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(timeStamp)
+      schemaEvolutionEntry.setRemoved(deletedColumnSchema.toList.asJava)
+      AlterTableUtil
+        .updateSchemaInfo(carbonTable,
+          schemaEvolutionEntry,
+          tableInfo)(sparkSession,
+          sparkSession.sessionState.asInstanceOf[CarbonSessionState])
+      // TODO: 1. add check for deletion of index tables
+      // delete dictionary files for dictionary column and clear dictionary cache from memory
+      new AlterTableDropColumnRDD(sparkSession.sparkContext,
+        dictionaryColumns,
+        carbonTable.getCarbonTableIdentifier,
+        carbonTable.getStorePath).collect()
+      LOGGER.info(s"Alter table for drop columns is successful for table $dbName.$tableName")
+      LOGGER.audit(s"Alter table for drop columns is successful for table $dbName.$tableName")
+    } catch {
+      case e: Exception => LOGGER
+        .error("Alter table drop columns failed : " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil.revertDropColumnChanges(dbName, tableName, timeStamp)(sparkSession)
+        }
+        sys.error(s"Alter table drop column operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+    }
+    Seq.empty
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
new file mode 100644
index 0000000..af361d5
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/AlterTableRenameTableCommand.scala
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.command.schema
+
+import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command.{AlterTableRenameModel, RunnableCommand}
+import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionState}
+import org.apache.spark.util.AlterTableUtil
+
+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.{ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+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.format.SchemaEvolutionEntry
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+private[sql] case class AlterTableRenameTableCommand(
+    alterTableRenameModel: AlterTableRenameModel)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val LOGGER: LogService = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+    val oldTableIdentifier = alterTableRenameModel.oldTableIdentifier
+    val newTableIdentifier = alterTableRenameModel.newTableIdentifier
+    val oldDatabaseName = oldTableIdentifier.database
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    val newDatabaseName = newTableIdentifier.database
+      .getOrElse(sparkSession.catalog.currentDatabase)
+    if (!oldDatabaseName.equalsIgnoreCase(newDatabaseName)) {
+      throw new MalformedCarbonCommandException("Database name should be same for both tables")
+    }
+    val tableExists = sparkSession.catalog.tableExists(oldDatabaseName, newTableIdentifier.table)
+    if (tableExists) {
+      throw new MalformedCarbonCommandException(s"Table with name $newTableIdentifier " +
+                                                s"already exists")
+    }
+    val oldTableName = oldTableIdentifier.table.toLowerCase
+    val newTableName = newTableIdentifier.table.toLowerCase
+    LOGGER.audit(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
+    LOGGER.info(s"Rename table request has been received for $oldDatabaseName.$oldTableName")
+    val metastore = CarbonEnv.getInstance(sparkSession).carbonMetastore
+    val relation: CarbonRelation =
+      metastore.lookupRelation(oldTableIdentifier.database, oldTableName)(sparkSession)
+        .asInstanceOf[CarbonRelation]
+    if (relation == null) {
+      LOGGER.audit(s"Rename table request has failed. " +
+                   s"Table $oldDatabaseName.$oldTableName does not exist")
+      sys.error(s"Table $oldDatabaseName.$oldTableName does not exist")
+    }
+    val locksToBeAcquired = List(LockUsage.METADATA_LOCK,
+      LockUsage.COMPACTION_LOCK,
+      LockUsage.DELETE_SEGMENT_LOCK,
+      LockUsage.CLEAN_FILES_LOCK,
+      LockUsage.DROP_TABLE_LOCK)
+    var locks = List.empty[ICarbonLock]
+    var timeStamp = 0L
+    var carbonTable: CarbonTable = null
+    try {
+      locks = AlterTableUtil
+        .validateTableAndAcquireLock(oldDatabaseName, oldTableName, locksToBeAcquired)(
+          sparkSession)
+      val tableMeta = metastore.lookupRelation(Some(oldDatabaseName), oldTableName)(sparkSession)
+        .asInstanceOf[CarbonRelation].tableMeta
+      carbonTable = tableMeta.carbonTable
+      // get the latest carbon table and check for column existence
+      val carbonTablePath = CarbonStorePath.
+        getCarbonTablePath(AbsoluteTableIdentifier.fromTablePath(tableMeta.tablePath))
+      val tableMetadataFile = carbonTablePath.getPath
+      val tableInfo: org.apache.carbondata.format.TableInfo =
+        metastore.getThriftTableInfo(carbonTablePath)(sparkSession)
+      val schemaEvolutionEntry = new SchemaEvolutionEntry(System.currentTimeMillis)
+      schemaEvolutionEntry.setTableName(newTableName)
+      timeStamp = System.currentTimeMillis()
+      schemaEvolutionEntry.setTime_stamp(timeStamp)
+      renameBadRecords(oldTableName, newTableName, oldDatabaseName)
+      val fileType = FileFactory.getFileType(tableMetadataFile)
+      if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
+        val rename = FileFactory.getCarbonFile(carbonTablePath.getPath, fileType)
+          .renameForce(carbonTablePath.getParent.toString + CarbonCommonConstants.FILE_SEPARATOR +
+                       newTableName)
+        if (!rename) {
+          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
+          sys.error(s"Folder rename failed for table $oldDatabaseName.$oldTableName")
+        }
+      }
+      val newTableIdentifier = new CarbonTableIdentifier(oldDatabaseName,
+        newTableName, carbonTable.getCarbonTableIdentifier.getTableId)
+      val newTablePath = metastore.updateTableSchema(newTableIdentifier,
+        carbonTable.getCarbonTableIdentifier,
+        tableInfo,
+        schemaEvolutionEntry,
+        tableMeta.tablePath)(sparkSession)
+      metastore.removeTableFromMetadata(oldDatabaseName, oldTableName)
+      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
+        .runSqlHive(
+          s"ALTER TABLE $oldDatabaseName.$oldTableName RENAME TO $oldDatabaseName.$newTableName")
+      sparkSession.sessionState.asInstanceOf[CarbonSessionState].metadataHive
+        .runSqlHive(
+          s"ALTER TABLE $oldDatabaseName.$newTableName SET SERDEPROPERTIES" +
+          s"('tableName'='$newTableName', " +
+          s"'dbName'='$oldDatabaseName', 'tablePath'='$newTablePath')")
+      sparkSession.catalog.refreshTable(TableIdentifier(newTableName,
+        Some(oldDatabaseName)).quotedString)
+      LOGGER.audit(s"Table $oldTableName has been successfully renamed to $newTableName")
+      LOGGER.info(s"Table $oldTableName has been successfully renamed to $newTableName")
+    } catch {
+      case e: Exception =>
+        LOGGER.error(e, "Rename table failed: " + e.getMessage)
+        if (carbonTable != null) {
+          AlterTableUtil
+            .revertRenameTableChanges(oldTableIdentifier,
+              newTableName,
+              carbonTable.getStorePath,
+              carbonTable.getCarbonTableIdentifier.getTableId,
+              timeStamp)(
+              sparkSession)
+          renameBadRecords(newTableName, oldTableName, oldDatabaseName)
+        }
+        sys.error(s"Alter table rename table operation failed: ${e.getMessage}")
+    } finally {
+      // release lock after command execution completion
+      AlterTableUtil.releaseLocks(locks)
+      // case specific to rename table as after table rename old table path will not be found
+      if (carbonTable != null) {
+        AlterTableUtil
+          .releaseLocksManually(locks,
+            locksToBeAcquired,
+            oldDatabaseName,
+            newTableName,
+            carbonTable.getStorePath)
+      }
+    }
+    Seq.empty
+  }
+
+  private def renameBadRecords(
+      oldTableName: String,
+      newTableName: String,
+      dataBaseName: String): Unit = {
+    val oldPath = CarbonUtil
+      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + oldTableName)
+    val newPath = CarbonUtil
+      .getBadLogPath(dataBaseName + CarbonCommonConstants.FILE_SEPARATOR + newTableName)
+    val fileType = FileFactory.getFileType(oldPath)
+    if (FileFactory.isFileExist(oldPath, fileType)) {
+      val renameSuccess = FileFactory.getCarbonFile(oldPath, fileType)
+        .renameForce(newPath)
+      if (!renameSuccess) {
+        sys.error(s"BadRecords Folder Rename Failed for table $dataBaseName.$oldTableName")
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
new file mode 100644
index 0000000..c6ca950
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -0,0 +1,554 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.strategy
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions.{Attribute, _}
+import org.apache.spark.sql.catalyst.planning.PhysicalOperation
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.optimizer.CarbonDecoderRelation
+import org.apache.spark.sql.sources.{BaseRelation, Filter}
+import org.apache.spark.sql.types.{AtomicType, IntegerType, StringType}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.schema.BucketingInfo
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.CarbonAliasDecoderRelation
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Carbon specific optimization for late decode (convert dictionary key to value as late as
+ * possible), which can improve the aggregation performance and reduce memory usage
+ */
+private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
+  val PUSHED_FILTERS = "PushedFilters"
+
+  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+    plan match {
+      case PhysicalOperation(projects, filters, l: LogicalRelation)
+        if l.relation.isInstanceOf[CarbonDatasourceHadoopRelation] =>
+        val relation = l.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+        pruneFilterProject(
+          l,
+          projects,
+          filters,
+          (a, f, needDecoder) => toCatalystRDD(l, a, relation.buildScan(
+            a.map(_.name).toArray, f), needDecoder)) :: Nil
+      case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
+        if ((profile.isInstanceOf[IncludeProfile] && profile.isEmpty) ||
+            !CarbonDictionaryDecoder.
+              isRequiredToDecode(CarbonDictionaryDecoder.
+                getDictionaryColumnMapping(child.output, relations, profile, aliasMap))) {
+          planLater(child) :: Nil
+        } else {
+          CarbonDictionaryDecoder(relations,
+            profile,
+            aliasMap,
+            planLater(child),
+            SparkSession.getActiveSession.get
+          ) :: Nil
+        }
+      case _ => Nil
+    }
+  }
+
+
+  def getDecoderRDD(
+      logicalRelation: LogicalRelation,
+      projectExprsNeedToDecode: ArrayBuffer[AttributeReference],
+      rdd: RDD[InternalRow],
+      output: Seq[Attribute]): RDD[InternalRow] = {
+    val table = logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+    val relation = CarbonDecoderRelation(logicalRelation.attributeMap,
+      logicalRelation.relation.asInstanceOf[CarbonDatasourceHadoopRelation])
+    val attrs = projectExprsNeedToDecode.map { attr =>
+      val newAttr = AttributeReference(attr.name,
+        attr.dataType,
+        attr.nullable,
+        attr.metadata)(attr.exprId, Option(table.carbonRelation.tableName))
+      relation.addAttribute(newAttr)
+      newAttr
+    }
+
+    new CarbonDecoderRDD(
+      Seq(relation),
+      IncludeProfile(attrs),
+      CarbonAliasDecoderRelation(),
+      rdd,
+      output,
+      CarbonEnv.getInstance(SparkSession.getActiveSession.get).storePath,
+      table.carbonTable.getTableInfo.serialize())
+  }
+
+  private[this] def toCatalystRDD(
+      relation: LogicalRelation,
+      output: Seq[Attribute],
+      rdd: RDD[InternalRow],
+      needDecode: ArrayBuffer[AttributeReference]):
+  RDD[InternalRow] = {
+    if (needDecode.nonEmpty) {
+      rdd.asInstanceOf[CarbonScanRDD].setVectorReaderSupport(false)
+      getDecoderRDD(relation, needDecode, rdd, output)
+    } else {
+      rdd.asInstanceOf[CarbonScanRDD]
+        .setVectorReaderSupport(supportBatchedDataSource(relation.relation.sqlContext, output))
+      rdd
+    }
+  }
+
+  protected def pruneFilterProject(
+      relation: LogicalRelation,
+      projects: Seq[NamedExpression],
+      filterPredicates: Seq[Expression],
+      scanBuilder: (Seq[Attribute], Array[Filter],
+        ArrayBuffer[AttributeReference]) => RDD[InternalRow]) = {
+    pruneFilterProjectRaw(
+      relation,
+      projects,
+      filterPredicates,
+      (requestedColumns, _, pushedFilters, a) => {
+        scanBuilder(requestedColumns, pushedFilters.toArray, a)
+      })
+  }
+
+  protected def pruneFilterProjectRaw(
+      relation: LogicalRelation,
+      rawProjects: Seq[NamedExpression],
+      filterPredicates: Seq[Expression],
+      scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
+        ArrayBuffer[AttributeReference]) => RDD[InternalRow]) = {
+    val projects = rawProjects.map {p =>
+      p.transform {
+        case CustomDeterministicExpression(exp) => exp
+      }
+    }.asInstanceOf[Seq[NamedExpression]]
+
+    val projectSet = AttributeSet(projects.flatMap(_.references))
+    val filterSet = AttributeSet(filterPredicates.flatMap(_.references))
+
+    val candidatePredicates = filterPredicates.map {
+      _ transform {
+        case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes.
+      }
+    }
+
+    val (unhandledPredicates, pushedFilters) =
+      selectFilters(relation.relation, candidatePredicates)
+
+    // A set of column attributes that are only referenced by pushed down filters.  We can eliminate
+    // them from requested columns.
+    val handledSet = {
+      val handledPredicates = filterPredicates.filterNot(unhandledPredicates.contains)
+      val unhandledSet = AttributeSet(unhandledPredicates.flatMap(_.references))
+      AttributeSet(handledPredicates.flatMap(_.references)) --
+      (projectSet ++ unhandledSet).map(relation.attributeMap)
+    }
+
+    // Combines all Catalyst filter `Expression`s that are either not convertible to data source
+    // `Filter`s or cannot be handled by `relation`.
+    val filterCondition = unhandledPredicates.reduceLeftOption(expressions.And)
+    val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+    val map = table.carbonRelation.metaData.dictionaryMap
+
+    val metadata: Map[String, String] = {
+      val pairs = ArrayBuffer.empty[(String, String)]
+
+      if (pushedFilters.nonEmpty) {
+        pairs += (PUSHED_FILTERS -> pushedFilters.mkString("[", ", ", "]"))
+      }
+      pairs.toMap
+    }
+
+
+    val needDecoder = ArrayBuffer[AttributeReference]()
+    filterCondition match {
+      case Some(exp: Expression) =>
+        exp.references.collect {
+          case attr: AttributeReference =>
+            val dict = map.get(attr.name)
+            if (dict.isDefined && dict.get) {
+              needDecoder += attr
+            }
+        }
+      case None =>
+    }
+
+    projects.map {
+      case attr: AttributeReference =>
+      case Alias(attr: AttributeReference, _) =>
+      case others =>
+        others.references.map { f =>
+          val dictionary = map.get(f.name)
+          if (dictionary.isDefined && dictionary.get) {
+            needDecoder += f.asInstanceOf[AttributeReference]
+          }
+        }
+    }
+
+    if (projects.map(_.toAttribute) == projects &&
+        projectSet.size == projects.size &&
+        filterSet.subsetOf(projectSet)) {
+      // When it is possible to just use column pruning to get the right projection and
+      // when the columns of this projection are enough to evaluate all filter conditions,
+      // just do a scan followed by a filter, with no extra project.
+      val requestedColumns = projects
+        // Safe due to if above.
+        .asInstanceOf[Seq[Attribute]]
+        // Match original case of attributes.
+        .map(relation.attributeMap)
+        // Don't request columns that are only referenced by pushed filters.
+        .filterNot(handledSet.contains)
+      val updateRequestedColumns = updateRequestedColumnsFunc(requestedColumns, table, needDecoder)
+
+      val updateProject = projects.map { expr =>
+        var attr = expr.toAttribute.asInstanceOf[AttributeReference]
+        if (!needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
+          val dict = map.get(attr.name)
+          if (dict.isDefined && dict.get) {
+            attr = AttributeReference(attr.name, IntegerType, attr.nullable, attr.metadata)(attr
+              .exprId, attr.qualifier)
+          }
+        }
+        attr
+      }
+      val scan = getDataSourceScan(relation,
+        updateProject,
+        scanBuilder,
+        candidatePredicates,
+        pushedFilters,
+        metadata,
+        needDecoder,
+        updateRequestedColumns.asInstanceOf[Seq[Attribute]])
+      filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan)
+    } else {
+
+      var newProjectList: Seq[Attribute] = Seq.empty
+      val updatedProjects = projects.map {
+          case a@Alias(s: ScalaUDF, name)
+            if name.equalsIgnoreCase(CarbonCommonConstants.POSITION_ID) ||
+                name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID) =>
+            val reference = AttributeReference(name, StringType, true)().withExprId(a.exprId)
+            newProjectList :+= reference
+            reference
+          case a@Alias(s: ScalaUDF, name)
+            if name.equalsIgnoreCase(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_SEGMENTID) =>
+            val reference =
+              AttributeReference(CarbonCommonConstants.CARBON_IMPLICIT_COLUMN_TUPLEID,
+                StringType, true)().withExprId(a.exprId)
+            newProjectList :+= reference
+            a.transform {
+              case s: ScalaUDF =>
+                ScalaUDF(s.function, s.dataType, Seq(reference), s.inputTypes)
+            }
+          case other => other
+      }
+      // Don't request columns that are only referenced by pushed filters.
+      val requestedColumns =
+        (projectSet ++ filterSet -- handledSet).map(relation.attributeMap).toSeq ++ newProjectList
+      val updateRequestedColumns = updateRequestedColumnsFunc(requestedColumns, table, needDecoder)
+      val scan = getDataSourceScan(relation,
+        updateRequestedColumns.asInstanceOf[Seq[Attribute]],
+        scanBuilder,
+        candidatePredicates,
+        pushedFilters,
+        metadata,
+        needDecoder,
+        updateRequestedColumns.asInstanceOf[Seq[Attribute]])
+      execution.ProjectExec(
+        updateRequestedColumnsFunc(updatedProjects, table,
+          needDecoder).asInstanceOf[Seq[NamedExpression]],
+        filterCondition.map(execution.FilterExec(_, scan)).getOrElse(scan))
+    }
+  }
+
+  def getDataSourceScan(relation: LogicalRelation,
+      output: Seq[Attribute],
+      scanBuilder: (Seq[Attribute], Seq[Expression], Seq[Filter],
+        ArrayBuffer[AttributeReference]) => RDD[InternalRow],
+      candidatePredicates: Seq[Expression],
+      pushedFilters: Seq[Filter],
+      metadata: Map[String, String],
+      needDecoder: ArrayBuffer[AttributeReference],
+      updateRequestedColumns: Seq[Attribute]): DataSourceScanExec = {
+    val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation]
+    if (supportBatchedDataSource(relation.relation.sqlContext, updateRequestedColumns) &&
+        needDecoder.isEmpty) {
+      BatchedDataSourceScanExec(
+        output,
+        scanBuilder(updateRequestedColumns, candidatePredicates, pushedFilters, needDecoder),
+        relation.relation,
+        getPartitioning(table.carbonTable, updateRequestedColumns),
+        metadata,
+        relation.catalogTable.map(_.identifier))
+    } else {
+      RowDataSourceScanExec(output,
+        scanBuilder(updateRequestedColumns, candidatePredicates, pushedFilters, needDecoder),
+        relation.relation,
+        getPartitioning(table.carbonTable, updateRequestedColumns),
+        metadata,
+        relation.catalogTable.map(_.identifier))
+    }
+  }
+
+  def updateRequestedColumnsFunc(requestedColumns: Seq[Expression],
+      relation: CarbonDatasourceHadoopRelation,
+      needDecoder: ArrayBuffer[AttributeReference]): Seq[Expression] = {
+    val map = relation.carbonRelation.metaData.dictionaryMap
+    requestedColumns.map {
+      case attr: AttributeReference =>
+        if (needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
+          attr
+        } else {
+          val dict = map.get(attr.name)
+          if (dict.isDefined && dict.get) {
+            AttributeReference(attr.name,
+              IntegerType,
+              attr.nullable,
+              attr.metadata)(attr.exprId, attr.qualifier)
+          } else {
+            attr
+          }
+        }
+      case alias @ Alias(attr: AttributeReference, name) =>
+        if (needDecoder.exists(_.name.equalsIgnoreCase(attr.name))) {
+          alias
+        } else {
+          val dict = map.get(attr.name)
+          if (dict.isDefined && dict.get) {
+            alias.transform {
+              case attrLocal: AttributeReference =>
+                AttributeReference(attr.name,
+                  IntegerType,
+                  attr.nullable,
+                  attr.metadata)(attr.exprId, attr.qualifier)
+            }
+          } else {
+            alias
+          }
+        }
+      case others => others
+    }
+  }
+
+  private def getPartitioning(carbonTable: CarbonTable,
+      output: Seq[Attribute]): Partitioning = {
+    val info: BucketingInfo = carbonTable.getBucketingInfo(carbonTable.getFactTableName)
+    if (info != null) {
+      val cols = info.getListOfColumns.asScala
+      val sortColumn = carbonTable.
+        getDimensionByTableName(carbonTable.getFactTableName).get(0).getColName
+      val numBuckets = info.getNumberOfBuckets
+      val bucketColumns = cols.flatMap { n =>
+        val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
+        attrRef match {
+          case Some(attr: AttributeReference) =>
+            Some(AttributeReference(attr.name,
+              CarbonScalaUtil.convertCarbonToSparkDataType(n.getDataType),
+              attr.nullable,
+              attr.metadata)(attr.exprId, attr.qualifier))
+          case _ => None
+        }
+      }
+      if (bucketColumns.size == cols.size) {
+        HashPartitioning(bucketColumns, numBuckets)
+      } else {
+        UnknownPartitioning(0)
+      }
+    } else {
+      UnknownPartitioning(0)
+    }
+  }
+
+  protected[sql] def selectFilters(
+      relation: BaseRelation,
+      predicates: Seq[Expression]): (Seq[Expression], Seq[Filter]) = {
+
+    // For conciseness, all Catalyst filter expressions of type `expressions.Expression` below are
+    // called `predicate`s, while all data source filters of type `sources.Filter` are simply called
+    // `filter`s.
+
+    val translated: Seq[(Expression, Filter)] =
+      for {
+        predicate <- predicates
+        filter <- translateFilter(predicate)
+      } yield predicate -> filter
+
+    // A map from original Catalyst expressions to corresponding translated data source filters.
+    val translatedMap: Map[Expression, Filter] = translated.toMap
+
+    // Catalyst predicate expressions that cannot be translated to data source filters.
+    val unrecognizedPredicates = predicates.filterNot(translatedMap.contains)
+
+    // Data source filters that cannot be handled by `relation`. The semantic of a unhandled filter
+    // at here is that a data source may not be able to apply this filter to every row
+    // of the underlying dataset.
+    val unhandledFilters = relation.unhandledFilters(translatedMap.values.toArray).toSet
+
+    val (unhandled, handled) = translated.partition {
+      case (predicate, filter) =>
+        unhandledFilters.contains(filter)
+    }
+
+    // Catalyst predicate expressions that can be translated to data source filters, but cannot be
+    // handled by `relation`.
+    val (unhandledPredicates, _) = unhandled.unzip
+
+    // Translated data source filters that can be handled by `relation`
+    val (_, handledFilters) = handled.unzip
+
+    // translated contains all filters that have been converted to the public Filter interface.
+    // We should always push them to the data source no matter whether the data source can apply
+    // a filter to every row or not.
+    val (_, translatedFilters) = translated.unzip
+
+    (unrecognizedPredicates ++ unhandledPredicates, translatedFilters)
+  }
+
+
+  /**
+   * Tries to translate a Catalyst [[Expression]] into data source [[Filter]].
+   *
+   * @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`.
+   */
+  protected[sql] def translateFilter(predicate: Expression, or: Boolean = false): Option[Filter] = {
+    predicate match {
+      case or@Or(left, right) =>
+
+        val leftFilter = translateFilter(left, true)
+        val rightFilter = translateFilter(right, true)
+        if (leftFilter.isDefined && rightFilter.isDefined) {
+          Some(sources.Or(leftFilter.get, rightFilter.get))
+        } else {
+          None
+        }
+
+      case And(left, right) =>
+        val leftFilter = translateFilter(left, or)
+        val rightFilter = translateFilter(right, or)
+        if (or) {
+          if (leftFilter.isDefined && rightFilter.isDefined) {
+            (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
+          } else {
+            None
+          }
+        } else {
+          (translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
+        }
+      case EqualTo(a: Attribute, Literal(v, t)) =>
+        Some(sources.EqualTo(a.name, v))
+      case EqualTo(l@Literal(v, t), a: Attribute) =>
+        Some(sources.EqualTo(a.name, v))
+      case c@EqualTo(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@EqualTo(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case Not(EqualTo(a: Attribute, Literal(v, t))) =>
+        Some(sources.Not(sources.EqualTo(a.name, v)))
+      case Not(EqualTo(Literal(v, t), a: Attribute)) =>
+        Some(sources.Not(sources.EqualTo(a.name, v)))
+      case c@Not(EqualTo(Cast(a: Attribute, _), Literal(v, t))) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@Not(EqualTo(Literal(v, t), Cast(a: Attribute, _))) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case IsNotNull(a: Attribute) => Some(sources.IsNotNull(a.name))
+      case IsNull(a: Attribute) => Some(sources.IsNull(a.name))
+      case Not(In(a: Attribute, list)) if !list.exists(!_.isInstanceOf[Literal]) =>
+        val hSet = list.map(e => e.eval(EmptyRow))
+        Some(sources.Not(sources.In(a.name, hSet.toArray)))
+      case In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) =>
+        val hSet = list.map(e => e.eval(EmptyRow))
+        Some(sources.In(a.name, hSet.toArray))
+      case c@Not(In(Cast(a: Attribute, _), list))
+        if !list.exists(!_.isInstanceOf[Literal]) =>
+        Some(CastExpr(c))
+      case c@In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
+        Some(CastExpr(c))
+      case InSet(a: Attribute, set) =>
+        Some(sources.In(a.name, set.toArray))
+      case Not(InSet(a: Attribute, set)) =>
+        Some(sources.Not(sources.In(a.name, set.toArray)))
+      case GreaterThan(a: Attribute, Literal(v, t)) =>
+        Some(sources.GreaterThan(a.name, v))
+      case GreaterThan(Literal(v, t), a: Attribute) =>
+        Some(sources.LessThan(a.name, v))
+      case c@GreaterThan(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@GreaterThan(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case LessThan(a: Attribute, Literal(v, t)) =>
+        Some(sources.LessThan(a.name, v))
+      case LessThan(Literal(v, t), a: Attribute) =>
+        Some(sources.GreaterThan(a.name, v))
+      case c@LessThan(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@LessThan(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case GreaterThanOrEqual(a: Attribute, Literal(v, t)) =>
+        Some(sources.GreaterThanOrEqual(a.name, v))
+      case GreaterThanOrEqual(Literal(v, t), a: Attribute) =>
+        Some(sources.LessThanOrEqual(a.name, v))
+      case c@GreaterThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@GreaterThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case LessThanOrEqual(a: Attribute, Literal(v, t)) =>
+        Some(sources.LessThanOrEqual(a.name, v))
+      case LessThanOrEqual(Literal(v, t), a: Attribute) =>
+        Some(sources.GreaterThanOrEqual(a.name, v))
+      case c@LessThanOrEqual(Cast(a: Attribute, _), Literal(v, t)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case c@LessThanOrEqual(Literal(v, t), Cast(a: Attribute, _)) =>
+        CastExpressionOptimization.checkIfCastCanBeRemove(c)
+      case StartsWith(a: Attribute, Literal(v, t)) =>
+        Some(sources.StringStartsWith(a.name, v.toString))
+      case c@EndsWith(a: Attribute, Literal(v, t)) =>
+        Some(CarbonEndsWith(c))
+      case c@Contains(a: Attribute, Literal(v, t)) =>
+        Some(CarbonContainsWith(c))
+      case others => None
+    }
+  }
+
+  def supportBatchedDataSource(sqlContext: SQLContext, cols: Seq[Attribute]): Boolean = {
+    val vectorizedReader = {
+      if (sqlContext.sparkSession.conf.contains(CarbonCommonConstants.ENABLE_VECTOR_READER)) {
+        sqlContext.sparkSession.conf.get(CarbonCommonConstants.ENABLE_VECTOR_READER)
+      } else if (System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER) != null) {
+        System.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER)
+      } else {
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER,
+          CarbonCommonConstants.ENABLE_VECTOR_READER_DEFAULT)
+      }
+    }
+    val supportCodegen =
+      sqlContext.conf.wholeStageEnabled && sqlContext.conf.wholeStageMaxNumFields >= cols.size
+    supportCodegen && vectorizedReader.toBoolean &&
+    cols.forall(_.dataType.isInstanceOf[AtomicType])
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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
new file mode 100644
index 0000000..715af1d
--- /dev/null
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.execution.strategy
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy}
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.execution.command.management.{AlterTableCompactionCommand, CarbonShowLoadsCommand, LoadTableByInsertCommand, LoadTableCommand}
+import org.apache.spark.sql.execution.command.partition.ShowCarbonPartitionsCommand
+import org.apache.spark.sql.execution.command.schema.{AlterTableAddColumnCommand, AlterTableDataTypeChangeCommand, AlterTableDropColumnCommand, AlterTableRenameTableCommand}
+import org.apache.spark.sql.hive.execution.command.{CarbonDropDatabaseCommand, CarbonResetCommand, CarbonSetCommand}
+
+import org.apache.carbondata.core.util.CarbonUtil
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
+/**
+ * Carbon strategies for ddl commands
+ */
+class DDLStrategy(sparkSession: SparkSession) extends SparkStrategy {
+
+  def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+    plan match {
+      case LoadDataCommand(identifier, path, isLocal, isOverwrite, partition)
+        if CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(identifier)(sparkSession) =>
+        ExecutedCommandExec(
+          LoadTableCommand(
+            identifier.database,
+            identifier.table.toLowerCase,
+            path,
+            Seq(),
+            Map(),
+            isOverwrite)) :: Nil
+      case alter@AlterTableRenameCommand(oldTableIdentifier, newTableIdentifier, _) =>
+        val dbOption = oldTableIdentifier.database.map(_.toLowerCase)
+        val tableIdentifier = TableIdentifier(oldTableIdentifier.table.toLowerCase(), dbOption)
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(tableIdentifier)(
+            sparkSession)
+        if (isCarbonTable) {
+          val renameModel = AlterTableRenameModel(tableIdentifier, newTableIdentifier)
+          ExecutedCommandExec(AlterTableRenameTableCommand(renameModel)) :: Nil
+        } else {
+          ExecutedCommandExec(alter) :: Nil
+        }
+      case DropTableCommand(identifier, ifNotExists, isView, _)
+        if CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .isTablePathExists(identifier)(sparkSession) =>
+        ExecutedCommandExec(
+          CarbonDropTableCommand(ifNotExists, identifier.database,
+            identifier.table.toLowerCase)) :: Nil
+      case ShowLoadsCommand(databaseName, table, limit) =>
+        ExecutedCommandExec(
+          CarbonShowLoadsCommand(
+            databaseName,
+            table.toLowerCase,
+            limit,
+            plan.output)) :: Nil
+      case InsertIntoCarbonTable(relation: CarbonDatasourceHadoopRelation,
+      _, child: LogicalPlan, overwrite, _) =>
+        ExecutedCommandExec(LoadTableByInsertCommand(relation, child, overwrite.enabled)) :: Nil
+      case createDb@CreateDatabaseCommand(dbName, ifNotExists, _, _, _) =>
+        CarbonUtil.createDatabaseDirectory(dbName, CarbonEnv.getInstance(sparkSession).storePath)
+        ExecutedCommandExec(createDb) :: Nil
+      case drop@DropDatabaseCommand(dbName, ifExists, isCascade) =>
+        ExecutedCommandExec(CarbonDropDatabaseCommand(drop)) :: Nil
+      case alterTable@AlterTableCompactionCommand(altertablemodel) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(TableIdentifier(altertablemodel.tableName,
+            altertablemodel.dbName))(sparkSession)
+        if (isCarbonTable) {
+          if (altertablemodel.compactionType.equalsIgnoreCase("minor") ||
+              altertablemodel.compactionType.equalsIgnoreCase("major")) {
+            ExecutedCommandExec(alterTable) :: Nil
+          } else {
+            throw new MalformedCarbonCommandException(
+              "Unsupported alter operation on carbon table")
+          }
+        } else {
+          throw new MalformedCarbonCommandException(
+            "Operation not allowed : " + altertablemodel.alterSql)
+        }
+      case dataTypeChange@AlterTableDataTypeChangeCommand(alterTableChangeDataTypeModel) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(TableIdentifier(alterTableChangeDataTypeModel.tableName,
+            alterTableChangeDataTypeModel.databaseName))(sparkSession)
+        if (isCarbonTable) {
+          ExecutedCommandExec(dataTypeChange) :: Nil
+        } else {
+          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
+        }
+      case addColumn@AlterTableAddColumnCommand(alterTableAddColumnsModel) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(TableIdentifier(alterTableAddColumnsModel.tableName,
+            alterTableAddColumnsModel.databaseName))(sparkSession)
+        if (isCarbonTable) {
+          ExecutedCommandExec(addColumn) :: Nil
+        } else {
+          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
+        }
+      case dropColumn@AlterTableDropColumnCommand(alterTableDropColumnModel) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(TableIdentifier(alterTableDropColumnModel.tableName,
+            alterTableDropColumnModel.databaseName))(sparkSession)
+        if (isCarbonTable) {
+          ExecutedCommandExec(dropColumn) :: Nil
+        } else {
+          throw new MalformedCarbonCommandException("Unsupported alter operation on hive table")
+        }
+      case desc@DescribeTableCommand(identifier, partitionSpec, isExtended, isFormatted)
+        if CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(identifier)(sparkSession) && isFormatted =>
+        val resolvedTable =
+          sparkSession.sessionState.executePlan(UnresolvedRelation(identifier, None)).analyzed
+        val resultPlan = sparkSession.sessionState.executePlan(resolvedTable).executedPlan
+        ExecutedCommandExec(
+          CarbonDescribeFormattedCommand(
+            resultPlan,
+            plan.output,
+            identifier)) :: Nil
+      case ShowPartitionsCommand(t, cols) =>
+        val isCarbonTable = CarbonEnv.getInstance(sparkSession).carbonMetastore
+          .tableExists(t)(sparkSession)
+        if (isCarbonTable) {
+          ExecutedCommandExec(ShowCarbonPartitionsCommand(t)) :: Nil
+        } else {
+          ExecutedCommandExec(ShowPartitionsCommand(t, cols)) :: Nil
+        }
+      case set@SetCommand(kv) =>
+        ExecutedCommandExec(CarbonSetCommand(set)) :: Nil
+      case reset@ResetCommand =>
+        ExecutedCommandExec(CarbonResetCommand()) :: Nil
+      case org.apache.spark.sql.execution.datasources.CreateTable(tableDesc, mode, None)
+        if tableDesc.provider.get != DDLUtils.HIVE_PROVIDER
+           && tableDesc.provider.get.equals("org.apache.spark.sql.CarbonSource") =>
+        val updatedCatalog =
+          CarbonSource.updateCatalogTableWithCarbonSchema(tableDesc, sparkSession)
+        val cmd =
+          CreateDataSourceTableCommand(updatedCatalog, ignoreIfExists = mode == SaveMode.Ignore)
+        ExecutedCommandExec(cmd) :: Nil
+      case _ => Nil
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
index c9fc46c..f61ab84 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
@@ -20,12 +20,12 @@ package org.apache.spark.sql.hive
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.CarbonTableIdentifierImplicit
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar}
-import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, NamedExpression}
 import org.apache.spark.sql.catalyst.plans.Inner
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.execution.{ProjectExec, SparkSqlParser, SubqueryExec}
-import org.apache.spark.sql.execution.command.ProjectForDeleteCommand
+import org.apache.spark.sql.execution.SparkSqlParser
+import org.apache.spark.sql.execution.command.mutation.ProjectForDeleteCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants
@@ -167,13 +167,13 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       updatedSelectPlan
     }
     val tid = CarbonTableIdentifierImplicit.toTableIdentifier(Seq(table.tableIdentifier.toString()))
-    val tidSeq = Seq(getDB.getDatabaseName(tid.database, sparkSession))
+    val tidSeq = Seq(GetDB.getDatabaseName(tid.database, sparkSession))
     val destinationTable = UnresolvedRelation(table.tableIdentifier, table.alias)
     ProjectForUpdate(destinationTable, columns, Seq(finalPlan))
   }
 
   def processDeleteRecordsQuery(selectStmt: String, table: UnresolvedRelation): LogicalPlan = {
-   val tidSeq = Seq(getDB.getDatabaseName(table.tableIdentifier.database, sparkSession),
+   val tidSeq = Seq(GetDB.getDatabaseName(table.tableIdentifier.database, sparkSession),
      table.tableIdentifier.table)
     var addedTupleId = false
     val parsePlan = parser.parsePlan(selectStmt)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
index 1fe6c83..478b178 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonSessionState.scala
@@ -25,16 +25,13 @@ import org.apache.spark.sql.catalyst.expressions.{PredicateSubquery, ScalarSubqu
 import org.apache.spark.sql.catalyst.optimizer.Optimizer
 import org.apache.spark.sql.catalyst.parser.ParserInterface
 import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, SubqueryAlias}
-import org.apache.spark.sql.execution.CarbonLateDecodeStrategy
-import org.apache.spark.sql.execution.command.DDLStrategy
-import org.apache.spark.sql.execution.datasources._
 import org.apache.spark.sql.execution.SparkOptimizer
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.execution.strategy.{CarbonLateDecodeStrategy, DDLStrategy}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.optimizer.CarbonLateDecodeRule
 import org.apache.spark.sql.parser.CarbonSparkSqlParser
 
-import org.apache.carbondata.processing.merger.TableMeta
-
 /**
  * This class will have carbon catalog and refresh the relation from cache if the carbontable in
  * carbon catalog is not same as cached carbon relation's carbon table

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/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 f435fa6..1d8bb8a 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
@@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.execution.command
 
 import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.Attribute
 import org.apache.spark.sql.execution.command._
 
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
@@ -26,7 +27,7 @@ import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
   extends RunnableCommand {
 
-  override val output = command.output
+  override val output: Seq[Attribute] = command.output
 
   override def run(sparkSession: SparkSession): Seq[Row] = {
     val dbName = command.databaseName
@@ -50,7 +51,7 @@ case class CarbonDropDatabaseCommand(command: DropDatabaseCommand)
 case class CarbonSetCommand(command: SetCommand)
   extends RunnableCommand {
 
-  override val output = command.output
+  override val output: Seq[Attribute] = command.output
 
   override def run(sparkSession: SparkSession): Seq[Row] = {
     val sessionParms = CarbonEnv.getInstance(sparkSession).carbonSessionInfo.getSessionParams

http://git-wip-us.apache.org/repos/asf/carbondata/blob/6627cac0/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
index c6dd905..2ddde7a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/optimizer/CarbonLateDecodeRule.scala
@@ -27,7 +27,8 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules.Rule
-import org.apache.spark.sql.execution.command.{ProjectForUpdateCommand, RunnableCommand}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.execution.command.mutation.ProjectForUpdateCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.types.{IntegerType, StringType}
 


[17/50] [abbrv] carbondata git commit: Updated new version number 1.3.0-SNAPSHOT in pom files

Posted by ja...@apache.org.
Updated new version number 1.3.0-SNAPSHOT in pom files


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

Branch: refs/heads/streaming_ingest
Commit: 87e45a4c592472a32efa3de559fb0ad69c245581
Parents: eb771f5
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Thu Sep 28 14:31:13 2017 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Thu Sep 28 14:31:13 2017 +0530

----------------------------------------------------------------------
 assembly/pom.xml                              | 2 +-
 common/pom.xml                                | 2 +-
 core/pom.xml                                  | 2 +-
 examples/flink/pom.xml                        | 2 +-
 examples/spark/pom.xml                        | 2 +-
 examples/spark2/pom.xml                       | 2 +-
 format/pom.xml                                | 2 +-
 hadoop/pom.xml                                | 2 +-
 integration/hive/pom.xml                      | 2 +-
 integration/presto/pom.xml                    | 2 +-
 integration/spark-common-cluster-test/pom.xml | 2 +-
 integration/spark-common-test/pom.xml         | 2 +-
 integration/spark-common/pom.xml              | 2 +-
 integration/spark/pom.xml                     | 2 +-
 integration/spark2/pom.xml                    | 2 +-
 pom.xml                                       | 2 +-
 processing/pom.xml                            | 2 +-
 17 files changed, 17 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index f4d4f70..d705b66 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index 8b2e002..6343361 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 1c6d0c4..50e6a6d 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/examples/flink/pom.xml
----------------------------------------------------------------------
diff --git a/examples/flink/pom.xml b/examples/flink/pom.xml
index d83e2cd..aa2a14f 100644
--- a/examples/flink/pom.xml
+++ b/examples/flink/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/examples/spark/pom.xml
----------------------------------------------------------------------
diff --git a/examples/spark/pom.xml b/examples/spark/pom.xml
index 051237c..cc078ff 100644
--- a/examples/spark/pom.xml
+++ b/examples/spark/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/examples/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/examples/spark2/pom.xml b/examples/spark2/pom.xml
index 1802451..94af8ec 100644
--- a/examples/spark2/pom.xml
+++ b/examples/spark2/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/format/pom.xml
----------------------------------------------------------------------
diff --git a/format/pom.xml b/format/pom.xml
index e57effd..ae84f0b 100644
--- a/format/pom.xml
+++ b/format/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop/pom.xml b/hadoop/pom.xml
index 4df8922..206246f 100644
--- a/hadoop/pom.xml
+++ b/hadoop/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/integration/hive/pom.xml
----------------------------------------------------------------------
diff --git a/integration/hive/pom.xml b/integration/hive/pom.xml
index 17a3cad..13bd581 100644
--- a/integration/hive/pom.xml
+++ b/integration/hive/pom.xml
@@ -22,7 +22,7 @@
     <parent>
         <groupId>org.apache.carbondata</groupId>
         <artifactId>carbondata-parent</artifactId>
-        <version>1.2.0-SNAPSHOT</version>
+        <version>1.3.0-SNAPSHOT</version>
         <relativePath>../../pom.xml</relativePath>
     </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/integration/presto/pom.xml
----------------------------------------------------------------------
diff --git a/integration/presto/pom.xml b/integration/presto/pom.xml
index 6a225f0..e11ce4c 100644
--- a/integration/presto/pom.xml
+++ b/integration/presto/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/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 0f71653..9728a5c 100644
--- a/integration/spark-common-cluster-test/pom.xml
+++ b/integration/spark-common-cluster-test/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/integration/spark-common-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/pom.xml b/integration/spark-common-test/pom.xml
index 29eae9e..7c67000 100644
--- a/integration/spark-common-test/pom.xml
+++ b/integration/spark-common-test/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/integration/spark-common/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common/pom.xml b/integration/spark-common/pom.xml
index 47760fc..82ff7a4 100644
--- a/integration/spark-common/pom.xml
+++ b/integration/spark-common/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/integration/spark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark/pom.xml b/integration/spark/pom.xml
index 3f23e1e..5060809 100644
--- a/integration/spark/pom.xml
+++ b/integration/spark/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index 1897339..a09bd62 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a1301f6..e7dd0e5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -35,7 +35,7 @@
   <inceptionYear>2016</inceptionYear>
   <packaging>pom</packaging>
 
-  <version>1.2.0-SNAPSHOT</version>
+  <version>1.3.0-SNAPSHOT</version>
 
   <licenses>
     <license>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/87e45a4c/processing/pom.xml
----------------------------------------------------------------------
diff --git a/processing/pom.xml b/processing/pom.xml
index d7e23f2..03c8b57 100644
--- a/processing/pom.xml
+++ b/processing/pom.xml
@@ -22,7 +22,7 @@
   <parent>
     <groupId>org.apache.carbondata</groupId>
     <artifactId>carbondata-parent</artifactId>
-    <version>1.2.0-SNAPSHOT</version>
+    <version>1.3.0-SNAPSHOT</version>
     <relativePath>../pom.xml</relativePath>
   </parent>
 


[03/50] [abbrv] carbondata git commit: [CARBONDATA-1448] fix partitionInfo null issue in CarbonTableInputFormat

Posted by ja...@apache.org.
[CARBONDATA-1448] fix partitionInfo null issue in CarbonTableInputFormat

This closes #1369


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

Branch: refs/heads/streaming_ingest
Commit: 80fa37c5547f7d19ee5471602ed16ca2b5303bcd
Parents: 1e7da59
Author: lionelcao <wh...@gmail.com>
Authored: Tue Sep 19 16:50:16 2017 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Fri Sep 22 11:39:15 2017 +0530

----------------------------------------------------------------------
 .../ThriftWrapperSchemaConverterImpl.java       |  4 +-
 .../core/metadata/schema/BucketingInfo.java     | 32 ++++++-
 .../core/metadata/schema/PartitionInfo.java     | 95 ++++++++++++++++++--
 .../core/metadata/schema/table/TableSchema.java | 25 ++++++
 .../src/test/resources/partition_data.csv       |  1 -
 .../partition/TestAlterPartitionTable.scala     | 69 +++++++-------
 6 files changed, 179 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/80fa37c5/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 2887d2a..bc877b7 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
@@ -219,7 +219,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     externalPartitionInfo.setList_info(wrapperPartitionInfo.getListInfo());
     externalPartitionInfo.setRange_info(wrapperPartitionInfo.getRangeInfo());
     externalPartitionInfo.setNum_partitions(wrapperPartitionInfo.getNumPartitions());
-    externalPartitionInfo.setMax_partition(wrapperPartitionInfo.getMAX_PARTITION());
+    externalPartitionInfo.setMax_partition(wrapperPartitionInfo.getMaxPartitionId());
     externalPartitionInfo.setPartition_ids(wrapperPartitionInfo
         .getPartitionIds());
     return externalPartitionInfo;
@@ -454,7 +454,7 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     wrapperPartitionInfo.setNumPartitions(externalPartitionInfo.getNum_partitions());
     wrapperPartitionInfo.setPartitionIds(externalPartitionInfo
         .getPartition_ids());
-    wrapperPartitionInfo.setMAX_PARTITION(externalPartitionInfo.getMax_partition());
+    wrapperPartitionInfo.setMaxPartitionId(externalPartitionInfo.getMax_partition());
     return wrapperPartitionInfo;
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80fa37c5/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 8434601..569241d 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
@@ -17,15 +17,20 @@
 
 package org.apache.carbondata.core.metadata.schema;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.List;
 
+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 {
+public class BucketingInfo implements Serializable, Writable {
 
   private static final long serialVersionUID = -0L;
 
@@ -33,6 +38,10 @@ public class BucketingInfo implements Serializable {
 
   private int numberOfBuckets;
 
+  public BucketingInfo() {
+
+  }
+
   public BucketingInfo(List<ColumnSchema> listOfColumns, int numberOfBuckets) {
     this.listOfColumns = listOfColumns;
     this.numberOfBuckets = numberOfBuckets;
@@ -46,4 +55,25 @@ public class BucketingInfo implements Serializable {
     return numberOfBuckets;
   }
 
+  @Override
+  public void write(DataOutput output) throws IOException {
+    output.writeInt(numberOfBuckets);
+    output.writeInt(listOfColumns.size());
+    for (ColumnSchema aColSchema : listOfColumns) {
+      aColSchema.write(output);
+    }
+  }
+
+  @Override
+  public void readFields(DataInput input) throws IOException {
+    this.numberOfBuckets = input.readInt();
+    int colSchemaSize = input.readInt();
+    this.listOfColumns = new ArrayList<>(colSchemaSize);
+    for (int i = 0; i < colSchemaSize; i++) {
+      ColumnSchema aSchema = new ColumnSchema();
+      aSchema.readFields(input);
+      this.listOfColumns.add(aSchema);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80fa37c5/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
index d0c4447..7a0e17f 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/PartitionInfo.java
@@ -17,17 +17,21 @@
 
 package org.apache.carbondata.core.metadata.schema;
 
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
+import org.apache.carbondata.core.metadata.schema.table.Writable;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 
 /**
  * Partition information of carbon partition table
  */
-public class PartitionInfo implements Serializable {
+public class PartitionInfo implements Serializable, Writable {
 
   private static final long serialVersionUID = -0L;
 
@@ -53,7 +57,7 @@ public class PartitionInfo implements Serializable {
   /**
    * current max partition id, increase only, will be used in alter table partition operation
    */
-  private int MAX_PARTITION;
+  private int maxPartitionId;
 
   /**
    * record the partitionId in the logical ascending order
@@ -61,6 +65,10 @@ public class PartitionInfo implements Serializable {
    */
   private List<Integer> partitionIds;
 
+  public PartitionInfo() {
+
+  }
+
   public PartitionInfo(List<ColumnSchema> columnSchemaList, PartitionType partitionType) {
     this.columnSchemaList = columnSchemaList;
     this.partitionType = partitionType;
@@ -72,7 +80,7 @@ public class PartitionInfo implements Serializable {
    */
   public void  addPartition(int addPartitionCount) {
     for (int i = 0; i < addPartitionCount; i++) {
-      partitionIds.add(++MAX_PARTITION);
+      partitionIds.add(++maxPartitionId);
       numPartitions++;
     }
   }
@@ -87,7 +95,7 @@ public class PartitionInfo implements Serializable {
   public void splitPartition(int sourcePartitionIndex, int newPartitionNumbers) {
     partitionIds.remove(sourcePartitionIndex);
     for (int i = 0; i < newPartitionNumbers; i++) {
-      partitionIds.add(sourcePartitionIndex + i, ++MAX_PARTITION);
+      partitionIds.add(sourcePartitionIndex + i, ++maxPartitionId);
     }
     numPartitions = numPartitions - 1 + newPartitionNumbers;
   }
@@ -129,7 +137,7 @@ public class PartitionInfo implements Serializable {
     for (int i = 0; i < partitionNum; i++) {
       partitionIds.add(i);
     }
-    MAX_PARTITION = partitionNum - 1;
+    maxPartitionId = partitionNum - 1;
     numPartitions = partitionNum;
   }
 
@@ -141,12 +149,12 @@ public class PartitionInfo implements Serializable {
     return numPartitions;
   }
 
-  public int getMAX_PARTITION() {
-    return MAX_PARTITION;
+  public int getMaxPartitionId() {
+    return maxPartitionId;
   }
 
-  public void setMAX_PARTITION(int max_partition) {
-    this.MAX_PARTITION = max_partition;
+  public void setMaxPartitionId(int maxPartitionId) {
+    this.maxPartitionId = maxPartitionId;
   }
 
   public List<Integer> getPartitionIds() {
@@ -161,4 +169,73 @@ public class PartitionInfo implements Serializable {
     return partitionIds.get(index);
   }
 
+  @Override
+  public void write(DataOutput output) throws IOException {
+    output.writeInt(columnSchemaList.size());
+    for (ColumnSchema columnSchema: columnSchemaList) {
+      columnSchema.write(output);
+    }
+    output.writeInt(partitionType.ordinal());
+    if (PartitionType.RANGE.equals(partitionType)) {
+      output.writeInt(rangeInfo.size());
+      for (String value: rangeInfo) {
+        output.writeUTF(value);
+      }
+    }
+    output.writeInt(partitionIds.size());
+    for (Integer value: partitionIds) {
+      output.writeInt(value);
+    }
+    if (PartitionType.LIST.equals(partitionType)) {
+      output.writeInt(listInfo.size());
+      for (List<String> listValue: listInfo) {
+        output.writeInt(listValue.size());
+        for (String value: listValue) {
+          output.writeUTF(value);
+        }
+      }
+    }
+    output.writeInt(numPartitions);
+    output.writeInt(maxPartitionId);
+  }
+
+  @Override
+  public void readFields(DataInput input) throws IOException {
+    int colSchemaSize = input.readInt();
+    this.columnSchemaList = new ArrayList<>(colSchemaSize);
+    for (int i = 0;i < colSchemaSize; i++) {
+      ColumnSchema colSchema = new ColumnSchema();
+      colSchema.readFields(input);
+      this.columnSchemaList.add(colSchema);
+    }
+    this.partitionType = PartitionType.values()[input.readInt()];
+    if (PartitionType.RANGE.equals(this.partitionType)) {
+      int rangeSize = input.readInt();
+      this.rangeInfo = new ArrayList<>(rangeSize);
+      for (int i = 0; i < rangeSize; i++) {
+        rangeInfo.add(input.readUTF());
+      }
+    }
+    int partitionIdSize = input.readInt();
+    partitionIds = new ArrayList<>(partitionIdSize);
+    for (int i = 0; i < partitionIdSize; i++) {
+      partitionIds.add(input.readInt());
+    }
+    if (PartitionType.LIST.equals(partitionType)) {
+      int listInfoSize = input.readInt();
+      int aListSize;
+      this.listInfo = new ArrayList<>(listInfoSize);
+      for (int i = 0; i < listInfoSize; i++) {
+        aListSize = input.readInt();
+        List<String> aList = new ArrayList<>(aListSize);
+        for (int j = 0; j < aListSize; j++) {
+          aList.add(input.readUTF());
+        }
+        this.listInfo.add(aList);
+      }
+    }
+
+    numPartitions = input.readInt();
+    maxPartitionId = input.readInt();
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80fa37c5/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 a396d19..7dc41a4 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
@@ -210,6 +210,19 @@ public class TableSchema implements Serializable, Writable {
     for (ColumnSchema column : listOfColumns) {
       column.write(out);
     }
+
+    if (null != partitionInfo) {
+      out.writeBoolean(true);
+      partitionInfo.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
+    if (null != bucketingInfo) {
+      out.writeBoolean(true);
+      bucketingInfo.write(out);
+    } else {
+      out.writeBoolean(false);
+    }
   }
 
   @Override
@@ -223,6 +236,18 @@ public class TableSchema implements Serializable, Writable {
       schema.readFields(in);
       this.listOfColumns.add(schema);
     }
+
+    boolean partitionExists = in.readBoolean();
+    if (partitionExists) {
+      this.partitionInfo = new PartitionInfo();
+      this.partitionInfo.readFields(in);
+    }
+
+    boolean bucketingExists = in.readBoolean();
+    if (bucketingExists) {
+      this.bucketingInfo = new BucketingInfo();
+      this.bucketingInfo.readFields(in);
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80fa37c5/integration/spark-common-test/src/test/resources/partition_data.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/partition_data.csv b/integration/spark-common-test/src/test/resources/partition_data.csv
index ec20d92..ae83c56 100644
--- a/integration/spark-common-test/src/test/resources/partition_data.csv
+++ b/integration/spark-common-test/src/test/resources/partition_data.csv
@@ -25,4 +25,3 @@ id,vin,logdate,phonenumber,country,area,salary
 24,A41158494839,2016/07/12,625371324,China,Asia,10023
 25,A41158494840,2016/07/12,626381324,Good,OutSpace,10024
 26,A41158494843,2016/07/12,625378824,NotGood,OutSpace,10025
-27,A41158494999,2016/07/12,625378824,Other,,10026

http://git-wip-us.apache.org/repos/asf/carbondata/blob/80fa37c5/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 29daac9..a4a431d 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
@@ -242,21 +242,21 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
 
   }
 
-  ignore("Alter table add partition: List Partition") {
+  test("Alter table add partition: List Partition") {
     sql("""ALTER TABLE list_table_area ADD PARTITION ('OutSpace', 'Hi')""".stripMargin)
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_list_table_area")
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     val partitionIds = partitionInfo.getPartitionIds
     val list_info = partitionInfo.getListInfo
     assert(partitionIds == List(0, 1, 2, 3, 4, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMAX_PARTITION == 5)
+    assert(partitionInfo.getMaxPartitionId == 5)
     assert(partitionInfo.getNumPartitions == 6)
     assert(list_info.get(0).get(0) == "Asia")
     assert(list_info.get(1).get(0) == "America")
     assert(list_info.get(2).get(0) == "Europe")
     assert(list_info.get(3).get(0) == "OutSpace")
     assert(list_info.get(4).get(0) == "Hi")
-    validateDataFiles("default_list_table_area", "0", Seq(0, 1, 2, 4))
+    validateDataFiles("default_list_table_area", "0", Seq(1, 2, 4))
     val result_after = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area")
     val result_origin = sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin")
     checkAnswer(result_after, result_origin)
@@ -290,13 +290,13 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds2 = partitionInfo2.getPartitionIds
     val list_info2 = partitionInfo2.getListInfo
     assert(partitionIds2 == List(0, 1, 3, 4, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo2.getMAX_PARTITION == 5)
+    assert(partitionInfo2.getMaxPartitionId == 5)
     assert(partitionInfo2.getNumPartitions == 5)
     assert(list_info2.get(0).get(0) == "Asia")
     assert(list_info2.get(1).get(0) == "Europe")
     assert(list_info2.get(2).get(0) == "OutSpace")
     assert(list_info2.get(3).get(0) == "Hi")
-    validateDataFiles("default_list_table_area", "0", Seq(0, 1, 4))
+    validateDataFiles("default_list_table_area", "0", Seq(1, 4))
     checkAnswer(sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area"),
       sql("select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <> 'America' "))
   }
@@ -308,7 +308,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds = partitionInfo.getPartitionIds
     val range_info = partitionInfo.getRangeInfo
     assert(partitionIds == List(0, 1, 2, 3, 4, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMAX_PARTITION == 5)
+    assert(partitionInfo.getMaxPartitionId == 5)
     assert(partitionInfo.getNumPartitions == 6)
     assert(range_info.get(0) == "2014/01/01")
     assert(range_info.get(1) == "2015/01/01")
@@ -346,7 +346,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds1 = partitionInfo1.getPartitionIds
     val range_info1 = partitionInfo1.getRangeInfo
     assert(partitionIds1 == List(0, 1, 2, 4, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMAX_PARTITION == 5)
+    assert(partitionInfo1.getMaxPartitionId == 5)
     assert(partitionInfo1.getNumPartitions == 5)
     assert(range_info1.get(0) == "2014/01/01")
     assert(range_info1.get(1) == "2015/01/01")
@@ -377,7 +377,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds = partitionInfo.getPartitionIds
     val list_info = partitionInfo.getListInfo
     assert(partitionIds == List(0, 1, 2, 3, 6, 7, 8, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMAX_PARTITION == 8)
+    assert(partitionInfo.getMaxPartitionId == 8)
     assert(partitionInfo.getNumPartitions == 8)
     assert(list_info.get(0).get(0) == "China")
     assert(list_info.get(0).get(1) == "US")
@@ -388,7 +388,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     assert(list_info.get(5).get(0) == "Good")
     assert(list_info.get(5).get(1) == "NotGood")
     assert(list_info.get(6).get(0) == "Korea")
-    validateDataFiles("default_list_table_country", "0", Seq(0, 1, 2, 3, 8))
+    validateDataFiles("default_list_table_country", "0", Seq(1, 2, 3, 8))
     val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country""")
     val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_country_origin""")
     checkAnswer(result_after, result_origin)
@@ -419,7 +419,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds1 = partitionInfo1.getPartitionIds
     val list_info1 = partitionInfo1.getListInfo
     assert(partitionIds1 == List(0, 1, 2, 3, 6, 7, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMAX_PARTITION == 8)
+    assert(partitionInfo1.getMaxPartitionId == 8)
     assert(partitionInfo1.getNumPartitions == 7)
     assert(list_info1.get(0).get(0) == "China")
     assert(list_info1.get(0).get(1) == "US")
@@ -442,7 +442,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds = partitionInfo.getPartitionIds
     val list_info = partitionInfo.getListInfo
     assert(partitionIds == List(0, 1, 2, 3, 6, 7, 5, 10, 11, 12).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMAX_PARTITION == 12)
+    assert(partitionInfo.getMaxPartitionId == 12)
     assert(partitionInfo.getNumPartitions == 10)
     assert(list_info.get(0).get(0) == "China")
     assert(list_info.get(0).get(1) == "US")
@@ -483,44 +483,45 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
 
   test("Alter table split partition with extra space in New SubList: List Partition") {
     sql("""ALTER TABLE list_table_area ADD PARTITION ('(One,Two, Three, Four)')""".stripMargin)
-    sql("""ALTER TABLE list_table_area SPLIT PARTITION(4) INTO ('One', '(Two, Three )', 'Four')""".stripMargin)
+    sql("""ALTER TABLE list_table_area SPLIT PARTITION(6) INTO ('One', '(Two, Three )', 'Four')""".stripMargin)
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_list_table_area")
     val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
     val partitionIds = partitionInfo.getPartitionIds
     val list_info = partitionInfo.getListInfo
-    assert(partitionIds == List(0, 1, 2, 3, 5, 6, 7).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMAX_PARTITION == 7)
-    assert(partitionInfo.getNumPartitions == 7)
+    assert(partitionIds == List(0, 1, 3, 4, 5, 7, 8, 9).map(Integer.valueOf(_)).asJava)
+    assert(partitionInfo.getMaxPartitionId == 9)
+    assert(partitionInfo.getNumPartitions == 8)
     assert(list_info.get(0).get(0) == "Asia")
-    assert(list_info.get(1).get(0) == "America")
-    assert(list_info.get(2).get(0) == "Europe")
-    assert(list_info.get(3).get(0) == "One")
-    assert(list_info.get(4).get(0) == "Two")
-    assert(list_info.get(4).get(1) == "Three")
-    assert(list_info.get(5).get(0) == "Four")
-    validateDataFiles("default_list_table_area", "0", Seq(0, 1, 2))
+    assert(list_info.get(1).get(0) == "Europe")
+    assert(list_info.get(2).get(0) == "OutSpace")
+    assert(list_info.get(3).get(0) == "Hi")
+    assert(list_info.get(4).get(0) == "One")
+    assert(list_info.get(5).get(0) == "Two")
+    assert(list_info.get(5).get(1) == "Three")
+    assert(list_info.get(6).get(0) == "Four")
+    validateDataFiles("default_list_table_area", "0", Seq(1, 4))
     val result_after = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area""")
-    val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin""")
+    val result_origin = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <> 'America' """)
     checkAnswer(result_after, result_origin)
 
     val result_after1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area < 'Four' """)
-    val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area < 'Four' """)
+    val result_origin1 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area < 'Four' and area <> 'America' """)
     checkAnswer(result_after1, result_origin1)
 
     val result_after2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area <= 'Four' """)
-    val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <= 'Four' """)
+    val result_origin2 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area <= 'Four' and area <> 'America'  """)
     checkAnswer(result_after2, result_origin2)
 
     val result_after3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area = 'Four' """)
-    val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area = 'Four' """)
+    val result_origin3 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area = 'Four' and area <> 'America'  """)
     checkAnswer(result_after3, result_origin3)
 
     val result_after4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area >= 'Four' """)
-    val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area >= 'Four' """)
+    val result_origin4 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area >= 'Four' and area <> 'America'  """)
     checkAnswer(result_after4, result_origin4)
 
     val result_after5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area where area > 'Four' """)
-    val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area > 'Four' """)
+    val result_origin5 = sql("""select id, vin, logdate, phonenumber, country, area, salary from list_table_area_origin where area > 'Four' and area <> 'America'  """)
     checkAnswer(result_after5, result_origin5)
   }
 
@@ -531,7 +532,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds = partitionInfo.getPartitionIds
     val rangeInfo = partitionInfo.getRangeInfo
     assert(partitionIds == List(0, 1, 2, 3, 5, 6).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMAX_PARTITION == 6)
+    assert(partitionInfo.getMaxPartitionId == 6)
     assert(partitionInfo.getNumPartitions == 6)
     assert(rangeInfo.get(0) == "2014/01/01")
     assert(rangeInfo.get(1) == "2015/01/01")
@@ -569,7 +570,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds1 = partitionInfo1.getPartitionIds
     val rangeInfo1 = partitionInfo1.getRangeInfo
     assert(partitionIds1 == List(0, 1, 2, 3, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMAX_PARTITION == 6)
+    assert(partitionInfo1.getMaxPartitionId == 6)
     assert(partitionInfo1.getNumPartitions == 5)
     assert(rangeInfo1.get(0) == "2014/01/01")
     assert(rangeInfo1.get(1) == "2015/01/01")
@@ -589,7 +590,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds = partitionInfo.getPartitionIds
     val rangeInfo = partitionInfo.getRangeInfo
     assert(partitionIds == List(0, 1, 2, 3, 5, 6).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo.getMAX_PARTITION == 6)
+    assert(partitionInfo.getMaxPartitionId == 6)
     assert(partitionInfo.getNumPartitions == 6)
     assert(rangeInfo.get(0) == "2014/01/01")
     assert(rangeInfo.get(1) == "2015/01/01")
@@ -627,7 +628,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds1 = partitionInfo1.getPartitionIds
     val rangeInfo1 = partitionInfo1.getRangeInfo
     assert(partitionIds1 == List(0, 1, 2, 3, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo1.getMAX_PARTITION == 6)
+    assert(partitionInfo1.getMaxPartitionId == 6)
     assert(partitionInfo1.getNumPartitions == 5)
     assert(rangeInfo1.get(0) == "2014/01/01")
     assert(rangeInfo1.get(1) == "2015/01/01")
@@ -645,7 +646,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds2 = partitionInfo2.getPartitionIds
     val rangeInfo2 = partitionInfo2.getRangeInfo
     assert(partitionIds2 == List(0, 1, 2, 5).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo2.getMAX_PARTITION == 6)
+    assert(partitionInfo2.getMaxPartitionId == 6)
     assert(partitionInfo2.getNumPartitions == 4)
     assert(rangeInfo2.get(0) == "2014/01/01")
     assert(rangeInfo2.get(1) == "2015/01/01")
@@ -662,7 +663,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
     val partitionIds3 = partitionInfo3.getPartitionIds
     val rangeInfo3 = partitionInfo3.getRangeInfo
     assert(partitionIds3 == List(0, 1, 2).map(Integer.valueOf(_)).asJava)
-    assert(partitionInfo3.getMAX_PARTITION == 6)
+    assert(partitionInfo3.getMaxPartitionId == 6)
     assert(partitionInfo3.getNumPartitions == 3)
     assert(rangeInfo3.get(0) == "2014/01/01")
     assert(rangeInfo3.get(1) == "2015/01/01")


[47/50] [abbrv] carbondata git commit: [CARBONDATA-1533] Fixed decimal data load fail issue and restricted max characters per column

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/ad25ffc3/integration/spark-common-test/src/test/resources/badrecords/bigtab.csv
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/resources/badrecords/bigtab.csv b/integration/spark-common-test/src/test/resources/badrecords/bigtab.csv
index 7518407..d9f44d8 100644
--- a/integration/spark-common-test/src/test/resources/badrecords/bigtab.csv
+++ b/integration/spark-common-test/src/test/resources/badrecords/bigtab.csv
@@ -1 +1 @@
-ckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagesp
 intobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobea
 nsarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequ
 icklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccounts
 detectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectf
 luffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffily
 accordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordi
 ngtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpre
 ssrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressreque
 sckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckages
 pintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobe
 ansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpress
 accountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccount
 sdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetect
 fluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffil
 yaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblith
 elyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpr
 essrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequ
 esckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckage
 spintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequickl
 yexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpres
 saccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccoun
 tsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetec
 tfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingto
 theblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblit
 helyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexp
 ressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressreq
 uesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansa
 requicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequick
 lyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpre
 ssaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluf
 filyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyacc
 ordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingt
 otheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtothebli
 thelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesck
 agespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespin
 tobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeans
 arequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequic
 klyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsde
 tectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectflu
 ffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyac
 cordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccording
 totheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpress
 requesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesc
 kagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespi
 ntobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobean
 sarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressac
 countsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsd
 etectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfl
 uffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilya
 ccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithel
 yexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpres
 srequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressreques
 ckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagesp
 intobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklye
 xpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressa
 ccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccounts
 detectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectf
 luffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtoth
 eblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithe
 lyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpre
 ssrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintob
 eansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansare
 quicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequickly
 expressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpress
 accountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffi
 lyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccor
 dingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtot
 heblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblith
 elyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithely6312,1
+equicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequickl
 yexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdete
 ctfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluff
 ilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyacco
 rdingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingto
 theblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressre
 quesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequescka
 gespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespint
 obeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpr
 essaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressacco
 untsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdet
 ectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluf
 filyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtothebl
 ithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelye
 xpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressr
 equesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesck
 agespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequi
 cklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexp
 ressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressacc
 ountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetectfluffilyaccordingtotheblithelyexpressrequesckagespintobeansarequicklyexpressaccountsdetect

<TRUNCATED>

[42/50] [abbrv] carbondata git commit: [CARBONDATA-1536] Default value of carbon.bad.records.action is FORCE

Posted by ja...@apache.org.
[CARBONDATA-1536] Default value of carbon.bad.records.action is FORCE

Default value of carbon.bad.records.action is 'FORCE' in source code, but the value is 'FAIL' in documentation. This PR fix documentation

This closes #1396


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

Branch: refs/heads/streaming_ingest
Commit: 7d3bf05030e4f3112aa487438307d3e93bf950c5
Parents: 349c59c
Author: xubo245 <60...@qq.com>
Authored: Thu Sep 28 23:15:21 2017 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Sun Oct 1 20:43:04 2017 +0800

----------------------------------------------------------------------
 docs/configuration-parameters.md                               | 2 +-
 docs/dml-operation-on-carbondata.md                            | 2 +-
 .../spark/testsuite/dataload/TestBatchSortDataLoad.scala       | 6 +-----
 3 files changed, 3 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7d3bf050/docs/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index 77320f6..e085317 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -60,7 +60,7 @@ This section provides the details of all the configurations required for CarbonD
 | carbon.update.persist.enable | true | Enabling this parameter considers persistent data. Enabling this will reduce the execution time of UPDATE operation. |  |
 | carbon.load.global.sort.partitions | 0 | The Number of partitions to use when shuffling data for sort. If user don't configurate or configurate it less than 1, it uses the number of map tasks as reduce tasks. In general, we recommend 2-3 tasks per CPU core in your cluster.
 | carbon.options.bad.records.logger.enable | false | Whether to create logs with details about bad records. | |
-| carbon.bad.records.action | fail | This property can have four types of actions for bad records FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. | |
+| carbon.bad.records.action | FORCE | This property can have four types of actions for bad records FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. | |
 | carbon.options.is.empty.data.bad.record | false | If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa. | |
 | carbon.options.bad.record.path |  | Specifies the HDFS path where bad records are stored. By default the value is Null. This path must to be configured by the user if bad record logger is enabled or bad record action redirect. | |
 | carbon.enable.vector.reader | true | This parameter increases the performance of select queries as it fetch columnar batch of size 4*1024 rows instead of fetching data row by row. | |

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7d3bf050/docs/dml-operation-on-carbondata.md
----------------------------------------------------------------------
diff --git a/docs/dml-operation-on-carbondata.md b/docs/dml-operation-on-carbondata.md
index 794bbc2..66109e8 100644
--- a/docs/dml-operation-on-carbondata.md
+++ b/docs/dml-operation-on-carbondata.md
@@ -198,7 +198,7 @@ OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true',
  | Options                   | Default Value | Description                                                                                                                                                                                                                                                                                                                                                                                                                                              |
  |---------------------------|---------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
  | BAD_RECORDS_LOGGER_ENABLE | false         | Whether to create logs with details about bad records.                                                                                                                                                                                                                                                                                                                                                                                                   |
- | BAD_RECORDS_ACTION        | FAIL          | Following are the four types of action for bad records:  FORCE: Auto-corrects the data by storing the bad records as NULL.  REDIRECT: Bad records are written to the raw CSV instead of being loaded.  IGNORE: Bad records are neither loaded nor written to the raw CSV.  FAIL: Data loading fails if any bad records are found.  NOTE: In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails. |
+ | BAD_RECORDS_ACTION        | FORCE          | Following are the four types of action for bad records:  FORCE: Auto-corrects the data by storing the bad records as NULL.  REDIRECT: Bad records are written to the raw CSV instead of being loaded.  IGNORE: Bad records are neither loaded nor written to the raw CSV.  FAIL: Data loading fails if any bad records are found.  NOTE: In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails. |
  | IS_EMPTY_DATA_BAD_RECORD  | false         | If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.                                                                                                                                                                                                                                                                                                                                                          |
  | BAD_RECORD_PATH           | -             | Specifies the HDFS path where bad records are stored. By default the value is Null. This path must to be configured by the user if bad record logger is enabled or bad record action redirect.                                                                                                                                                                                                                                                           |
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7d3bf050/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 46a4f8c..4c6c56d 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
@@ -61,15 +61,11 @@ class TestBatchSortDataLoad extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS carbon_load6")
   }
 
-
-
   override def beforeAll {
     dropTable
     buildTestData
   }
-
-
-
+  
   test("test batch sort load by passing option to load command") {
 
     sql(


[04/50] [abbrv] carbondata git commit: [CARBONDATA-1509] Fixed bug for maintaining compatibility of decimal type with older releases of Carbondata

Posted by ja...@apache.org.
[CARBONDATA-1509] Fixed bug for maintaining compatibility of decimal type with older releases of Carbondata

In old Carbondata releases, precision and scale is not stored for decimal data type and both values are initialized to -1. In TableSpec.ColumnSpec default values for precision and scale are initialized to 0 because of which exception is thrown while reading the old store with decimal column. Both precision and scale should be initialized to -1.

This closes #1378


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

Branch: refs/heads/streaming_ingest
Commit: 7f6b08a2b5b13799a2a0bbd29d40f45da14d07b1
Parents: 80fa37c
Author: manishgupta88 <to...@gmail.com>
Authored: Fri Sep 22 12:22:39 2017 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Fri Sep 22 16:28:05 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/carbondata/core/datastore/TableSpec.java     | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7f6b08a2/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
index 2fdf82b..b0b62db 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
@@ -130,7 +130,9 @@ public class TableSpec {
     }
 
     public ColumnSpec(String fieldName, DataType schemaDataType, ColumnType columnType) {
-      this(fieldName, schemaDataType, columnType, 0, 0);
+      // for backward compatibility as the precision and scale is not stored, the values should be
+      // initialized with -1 for both precision and scale
+      this(fieldName, schemaDataType, columnType, -1, -1);
     }
 
     public ColumnSpec(String fieldName, DataType schemaDataType, ColumnType columnType,


[33/50] [abbrv] carbondata git commit: [CARBONDATA-1530] Clean up carbon-processing module

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..1e73867
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorStepImpl.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.File;
+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.common.constants.LoggerAction;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
+import org.apache.carbondata.processing.loading.BadRecordsLogger;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+import org.apache.carbondata.processing.loading.converter.RowConverter;
+import org.apache.carbondata.processing.loading.converter.impl.RowConverterImpl;
+import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+/**
+ * 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 DataConverterProcessorStepImpl extends AbstractDataLoadProcessorStep {
+
+  private List<RowConverter> converters;
+  private BadRecordsLogger badRecordLogger;
+
+  public DataConverterProcessorStepImpl(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 = createBadRecordLogger(configuration);
+    RowConverter converter =
+        new RowConverterImpl(child.getOutput(), configuration, badRecordLogger);
+    configuration.setCardinalityFinder(converter);
+    converters.add(converter);
+    converter.initialize();
+  }
+
+  /**
+   * 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>() {
+      private boolean first = true;
+      private RowConverter localConverter;
+      @Override public boolean hasNext() {
+        if (first) {
+          first = false;
+          localConverter = converters.get(0).createCopyForNewThread();
+          synchronized (converters) {
+            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) {
+    CarbonRowBatch newBatch = new CarbonRowBatch(rowBatch.getSize());
+    while (rowBatch.hasNext()) {
+      newBatch.addRow(localConverter.convert(rowBatch.next()));
+    }
+    rowCounter.getAndAdd(newBatch.getSize());
+    return newBatch;
+  }
+
+  @Override
+  protected CarbonRow processRow(CarbonRow row) {
+    throw new UnsupportedOperationException();
+  }
+
+  public static BadRecordsLogger createBadRecordLogger(CarbonDataLoadConfiguration configuration) {
+    boolean badRecordsLogRedirect = false;
+    boolean badRecordConvertNullDisable = false;
+    boolean isDataLoadFail = false;
+    boolean badRecordsLoggerEnable = Boolean.parseBoolean(
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ENABLE)
+            .toString());
+    Object bad_records_action =
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ACTION)
+            .toString();
+    if (null != bad_records_action) {
+      LoggerAction loggerAction = null;
+      try {
+        loggerAction = LoggerAction.valueOf(bad_records_action.toString().toUpperCase());
+      } catch (IllegalArgumentException e) {
+        loggerAction = LoggerAction.FORCE;
+      }
+      switch (loggerAction) {
+        case FORCE:
+          badRecordConvertNullDisable = false;
+          break;
+        case REDIRECT:
+          badRecordsLogRedirect = true;
+          badRecordConvertNullDisable = true;
+          break;
+        case IGNORE:
+          badRecordsLogRedirect = false;
+          badRecordConvertNullDisable = true;
+          break;
+        case FAIL:
+          isDataLoadFail = true;
+          break;
+      }
+    }
+    CarbonTableIdentifier identifier =
+        configuration.getTableIdentifier().getCarbonTableIdentifier();
+    return new BadRecordsLogger(identifier.getBadRecordLoggerKey(),
+        identifier.getTableName() + '_' + System.currentTimeMillis(),
+        getBadLogStoreLocation(configuration,
+            identifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + identifier
+                .getTableName() + CarbonCommonConstants.FILE_SEPARATOR + configuration
+                .getSegmentId() + CarbonCommonConstants.FILE_SEPARATOR + configuration.getTaskNo()),
+        badRecordsLogRedirect, badRecordsLoggerEnable, badRecordConvertNullDisable, isDataLoadFail);
+  }
+
+  public static String getBadLogStoreLocation(CarbonDataLoadConfiguration configuration,
+      String storeLocation) {
+    String badLogStoreLocation = (String) configuration
+        .getDataLoadProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH);
+    if (null == badLogStoreLocation) {
+      badLogStoreLocation =
+          CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC);
+    }
+    badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation;
+
+    return badLogStoreLocation;
+  }
+
+  @Override
+  public void close() {
+    if (!closed) {
+      if (null != badRecordLogger) {
+        badRecordLogger.closeStreams();
+        renameBadRecord(badRecordLogger, configuration);
+      }
+      super.close();
+      if (converters != null) {
+        for (RowConverter converter : converters) {
+          if (null != converter) {
+            converter.finish();
+          }
+        }
+      }
+    }
+  }
+
+  public static void close(BadRecordsLogger badRecordLogger, CarbonDataLoadConfiguration
+      configuration, RowConverter converter) {
+    if (badRecordLogger != null) {
+      badRecordLogger.closeStreams();
+      renameBadRecord(badRecordLogger, configuration);
+    }
+    if (converter != null) {
+      converter.finish();
+    }
+  }
+
+  private static void renameBadRecord(BadRecordsLogger badRecordLogger,
+      CarbonDataLoadConfiguration configuration) {
+    // rename operation should be performed only in case either bad reccords loggers is enabled
+    // or bad records redirect is enabled
+    if (badRecordLogger.isBadRecordLoggerEnable() || badRecordLogger.isBadRecordsLogRedirect()) {
+      // rename the bad record in progress to normal
+      CarbonTableIdentifier identifier =
+          configuration.getTableIdentifier().getCarbonTableIdentifier();
+      CarbonDataProcessorUtil.renameBadRecordsFromInProgressToNormal(configuration,
+          identifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + identifier
+              .getTableName() + CarbonCommonConstants.FILE_SEPARATOR + configuration.getSegmentId()
+              + CarbonCommonConstants.FILE_SEPARATOR + configuration.getTaskNo());
+    }
+  }
+
+  @Override protected String getStepName() {
+    return "Data Converter";
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..009c6a0
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataConverterProcessorWithBucketingStepImpl.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.processing.loading.steps;
+
+import java.io.File;
+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.common.constants.LoggerAction;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.schema.BucketingInfo;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
+import org.apache.carbondata.processing.loading.BadRecordsLogger;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.constants.DataLoadProcessorConstants;
+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.CarbonDataProcessorUtil;
+
+/**
+ * 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 = createBadRecordLogger();
+    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) {
+    CarbonRowBatch newBatch = new CarbonRowBatch(rowBatch.getSize());
+    while (rowBatch.hasNext()) {
+      CarbonRow next = rowBatch.next();
+      short bucketNumber = (short) partitioner.getPartition(next.getData());
+      CarbonRow convertRow = localConverter.convert(next);
+      convertRow.bucketNumber = bucketNumber;
+      newBatch.addRow(convertRow);
+    }
+    rowCounter.getAndAdd(newBatch.getSize());
+    return newBatch;
+  }
+
+  @Override
+  protected CarbonRow processRow(CarbonRow row) {
+    throw new UnsupportedOperationException();
+  }
+
+  private BadRecordsLogger createBadRecordLogger() {
+    boolean badRecordsLogRedirect = false;
+    boolean badRecordConvertNullDisable = false;
+    boolean isDataLoadFail = false;
+    boolean badRecordsLoggerEnable = Boolean.parseBoolean(
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ENABLE)
+            .toString());
+    Object bad_records_action =
+        configuration.getDataLoadProperty(DataLoadProcessorConstants.BAD_RECORDS_LOGGER_ACTION)
+            .toString();
+    if (null != bad_records_action) {
+      LoggerAction loggerAction = null;
+      try {
+        loggerAction = LoggerAction.valueOf(bad_records_action.toString().toUpperCase());
+      } catch (IllegalArgumentException e) {
+        loggerAction = LoggerAction.FORCE;
+      }
+      switch (loggerAction) {
+        case FORCE:
+          badRecordConvertNullDisable = false;
+          break;
+        case REDIRECT:
+          badRecordsLogRedirect = true;
+          badRecordConvertNullDisable = true;
+          break;
+        case IGNORE:
+          badRecordsLogRedirect = false;
+          badRecordConvertNullDisable = true;
+          break;
+        case FAIL:
+          isDataLoadFail = true;
+          break;
+      }
+    }
+    CarbonTableIdentifier identifier =
+        configuration.getTableIdentifier().getCarbonTableIdentifier();
+    return new BadRecordsLogger(identifier.getBadRecordLoggerKey(),
+        identifier.getTableName() + '_' + System.currentTimeMillis(), getBadLogStoreLocation(
+        identifier.getDatabaseName() + CarbonCommonConstants.FILE_SEPARATOR + identifier
+            .getTableName() + CarbonCommonConstants.FILE_SEPARATOR + configuration.getSegmentId()
+            + CarbonCommonConstants.FILE_SEPARATOR + configuration.getTaskNo()),
+        badRecordsLogRedirect, badRecordsLoggerEnable, badRecordConvertNullDisable, isDataLoadFail);
+  }
+
+  private String getBadLogStoreLocation(String storeLocation) {
+    String badLogStoreLocation = (String) configuration
+        .getDataLoadProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH);
+    if (null == badLogStoreLocation) {
+      badLogStoreLocation =
+          CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC);
+    }
+    badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation;
+
+    return badLogStoreLocation;
+  }
+
+  @Override
+  public void close() {
+    if (!closed) {
+      super.close();
+      if (null != badRecordLogger) {
+        badRecordLogger.closeStreams();
+        renameBadRecord(configuration);
+      }
+      if (converters != null) {
+        for (RowConverter converter : converters) {
+          converter.finish();
+        }
+      }
+    }
+  }
+  private static void renameBadRecord(CarbonDataLoadConfiguration configuration) {
+    // rename the bad record in progress to normal
+    CarbonTableIdentifier identifier =
+        configuration.getTableIdentifier().getCarbonTableIdentifier();
+    CarbonDataProcessorUtil.renameBadRecordsFromInProgressToNormal(configuration,
+        identifier.getDatabaseName() + File.separator + identifier.getTableName()
+            + File.separator + configuration.getSegmentId() + File.separator + configuration
+            .getTaskNo());
+  }
+  @Override protected String getStepName() {
+    return "Data Converter with Bucketing";
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..f030d52
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterBatchProcessorStepImpl.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.processing.loading.steps;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+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.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
+import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.exception.BadRecordFoundException;
+import org.apache.carbondata.processing.loading.exception.CarbonDataLoadingException;
+import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
+import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
+import org.apache.carbondata.processing.store.CarbonFactHandler;
+import org.apache.carbondata.processing.store.CarbonFactHandlerFactory;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+/**
+ * It reads data from batch of sorted files(it could be in-memory/disk based files)
+ * which are generated in previous sort step. And it writes data to carbondata file.
+ * It also generates mdk key while writing to carbondata file
+ */
+public class DataWriterBatchProcessorStepImpl extends AbstractDataLoadProcessorStep {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DataWriterBatchProcessorStepImpl.class.getName());
+
+  public DataWriterBatchProcessorStepImpl(CarbonDataLoadConfiguration configuration,
+      AbstractDataLoadProcessorStep child) {
+    super(configuration, child);
+  }
+
+  @Override public DataField[] getOutput() {
+    return child.getOutput();
+  }
+
+  @Override public void initialize() throws IOException {
+    super.initialize();
+    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;
+  }
+
+  @Override public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
+    Iterator<CarbonRowBatch>[] iterators = child.execute();
+    CarbonTableIdentifier tableIdentifier =
+        configuration.getTableIdentifier().getCarbonTableIdentifier();
+    String tableName = tableIdentifier.getTableName();
+    try {
+      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+              System.currentTimeMillis());
+      int i = 0;
+      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++);
+            CarbonFactHandler dataHandler = CarbonFactHandlerFactory
+                .createCarbonFactHandler(model, CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
+            dataHandler.initialise();
+            processBatch(next, dataHandler);
+            finish(tableName, dataHandler);
+          }
+        }
+        i++;
+      }
+    } catch (Exception e) {
+      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterBatchProcessorStepImpl");
+      if (e.getCause() instanceof BadRecordFoundException) {
+        throw new BadRecordFoundException(e.getCause().getMessage());
+      }
+      throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage());
+    }
+    return null;
+  }
+
+  @Override protected String getStepName() {
+    return "Data Batch Writer";
+  }
+
+  private void finish(String tableName, CarbonFactHandler dataHandler) {
+    try {
+      dataHandler.finish();
+    } catch (Exception e) {
+      LOGGER.error(e, "Failed for table: " + tableName + " in  finishing data handler");
+    }
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
+    processingComplete(dataHandler);
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+            System.currentTimeMillis());
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
+  }
+
+  private void processingComplete(CarbonFactHandler dataHandler) {
+    if (null != dataHandler) {
+      try {
+        dataHandler.closeHandler();
+      } catch (Exception e) {
+        LOGGER.error(e);
+        throw new CarbonDataLoadingException(
+            "There is an unexpected error while closing data handler", e);
+      }
+    }
+  }
+
+  private void processBatch(CarbonRowBatch batch, CarbonFactHandler dataHandler) throws Exception {
+    int batchSize = 0;
+    while (batch.hasNext()) {
+      CarbonRow row = batch.next();
+      dataHandler.addDataToStore(row);
+      batchSize++;
+    }
+    batch.close();
+    rowCounter.getAndAdd(batchSize);
+  }
+
+  @Override protected CarbonRow processRow(CarbonRow row) {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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
new file mode 100644
index 0000000..e8e2b0e
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/DataWriterProcessorStepImpl.java
@@ -0,0 +1,199 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing.loading.steps;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+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.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.processing.loading.AbstractDataLoadProcessorStep;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+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.store.CarbonFactDataHandlerModel;
+import org.apache.carbondata.processing.store.CarbonFactHandler;
+import org.apache.carbondata.processing.store.CarbonFactHandlerFactory;
+import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
+
+/**
+ * It reads data from sorted files which are generated in previous sort step.
+ * And it writes data to carbondata file. It also generates mdk key while writing to carbondata file
+ */
+public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DataWriterProcessorStepImpl.class.getName());
+
+  private long readCounter;
+
+  public DataWriterProcessorStepImpl(CarbonDataLoadConfiguration configuration,
+      AbstractDataLoadProcessorStep child) {
+    super(configuration, child);
+  }
+
+  public DataWriterProcessorStepImpl(CarbonDataLoadConfiguration configuration) {
+    super(configuration, null);
+  }
+
+  @Override public DataField[] getOutput() {
+    return child.getOutput();
+  }
+
+  @Override public void initialize() throws IOException {
+    super.initialize();
+    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;
+  }
+
+  public CarbonFactDataHandlerModel getDataHandlerModel(int partitionId) {
+    CarbonTableIdentifier tableIdentifier =
+        configuration.getTableIdentifier().getCarbonTableIdentifier();
+    String[] storeLocation = getStoreLocation(tableIdentifier, String.valueOf(partitionId));
+    CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
+        .createCarbonFactDataHandlerModel(configuration, storeLocation, partitionId, 0);
+    return model;
+  }
+
+  @Override public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
+    Iterator<CarbonRowBatch>[] iterators = child.execute();
+    CarbonTableIdentifier tableIdentifier =
+        configuration.getTableIdentifier().getCarbonTableIdentifier();
+    String tableName = tableIdentifier.getTableName();
+    try {
+      CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+          .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+              System.currentTimeMillis());
+      int i = 0;
+      for (Iterator<CarbonRowBatch> iterator : iterators) {
+        String[] storeLocation = getStoreLocation(tableIdentifier, String.valueOf(i));
+
+        CarbonFactDataHandlerModel model = CarbonFactDataHandlerModel
+            .createCarbonFactDataHandlerModel(configuration, storeLocation, i, 0);
+        CarbonFactHandler dataHandler = null;
+        boolean rowsNotExist = true;
+        while (iterator.hasNext()) {
+          if (rowsNotExist) {
+            rowsNotExist = false;
+            dataHandler = CarbonFactHandlerFactory
+                .createCarbonFactHandler(model, CarbonFactHandlerFactory.FactHandlerType.COLUMNAR);
+            dataHandler.initialise();
+          }
+          processBatch(iterator.next(), dataHandler);
+        }
+        if (!rowsNotExist) {
+          finish(dataHandler);
+        }
+        i++;
+      }
+
+    } catch (CarbonDataWriterException e) {
+      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
+      throw new CarbonDataLoadingException(
+          "Error while initializing data handler : " + e.getMessage());
+    } catch (Exception e) {
+      LOGGER.error(e, "Failed for table: " + tableName + " in DataWriterProcessorStepImpl");
+      throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage(), e);
+    }
+    return null;
+  }
+
+  @Override protected String getStepName() {
+    return "Data Writer";
+  }
+
+  public void finish(CarbonFactHandler dataHandler) {
+    CarbonTableIdentifier tableIdentifier =
+        configuration.getTableIdentifier().getCarbonTableIdentifier();
+    String tableName = tableIdentifier.getTableName();
+
+    try {
+      dataHandler.finish();
+    } catch (Exception e) {
+      LOGGER.error(e, "Failed for table: " + tableName + " in  finishing data handler");
+    }
+    LOGGER.info("Record Processed For table: " + tableName);
+    String logMessage =
+        "Finished Carbon DataWriterProcessorStepImpl: Read: " + readCounter + ": Write: "
+            + rowCounter.get();
+    LOGGER.info(logMessage);
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(rowCounter.get());
+    processingComplete(dataHandler);
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+        .recordDictionaryValue2MdkAdd2FileTime(configuration.getPartitionId(),
+            System.currentTimeMillis());
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
+        .recordMdkGenerateTotalTime(configuration.getPartitionId(), System.currentTimeMillis());
+  }
+
+  private void processingComplete(CarbonFactHandler dataHandler) throws CarbonDataLoadingException {
+    if (null != dataHandler) {
+      try {
+        dataHandler.closeHandler();
+      } catch (CarbonDataWriterException e) {
+        LOGGER.error(e, e.getMessage());
+        throw new CarbonDataLoadingException(e.getMessage(), e);
+      } catch (Exception e) {
+        LOGGER.error(e, e.getMessage());
+        throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage());
+      }
+    }
+  }
+
+  private void processBatch(CarbonRowBatch batch, CarbonFactHandler dataHandler)
+      throws CarbonDataLoadingException {
+    try {
+      while (batch.hasNext()) {
+        CarbonRow row = batch.next();
+        dataHandler.addDataToStore(row);
+        readCounter++;
+      }
+    } catch (Exception e) {
+      throw new CarbonDataLoadingException(e);
+    }
+    rowCounter.getAndAdd(batch.getSize());
+  }
+
+  public void processRow(CarbonRow row, CarbonFactHandler dataHandler) throws KeyGenException {
+    try {
+      readCounter++;
+      dataHandler.addDataToStore(row);
+    } catch (Exception e) {
+      throw new CarbonDataLoadingException("unable to generate the mdkey", e);
+    }
+    rowCounter.getAndAdd(1);
+  }
+
+  @Override protected CarbonRow processRow(CarbonRow row) {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java
new file mode 100644
index 0000000..70a1254
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/InputProcessorStepImpl.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 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.atomic.AtomicLong;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+import org.apache.carbondata.processing.loading.DataField;
+import org.apache.carbondata.processing.loading.parser.RowParser;
+import org.apache.carbondata.processing.loading.parser.impl.RowParserImpl;
+import org.apache.carbondata.processing.loading.row.CarbonRowBatch;
+
+/**
+ * It reads data from record reader and sends data to next step.
+ */
+public class InputProcessorStepImpl extends AbstractDataLoadProcessorStep {
+
+  private RowParser rowParser;
+
+  private CarbonIterator<Object[]>[] inputIterators;
+
+  /**
+   * executor service to execute the query
+   */
+  public ExecutorService executorService;
+
+  public InputProcessorStepImpl(CarbonDataLoadConfiguration configuration,
+      CarbonIterator<Object[]>[] inputIterators) {
+    super(configuration, null);
+    this.inputIterators = inputIterators;
+  }
+
+  @Override public DataField[] getOutput() {
+    return configuration.getDataFields();
+  }
+
+  @Override public void initialize() throws IOException {
+    super.initialize();
+    rowParser = new RowParserImpl(getOutput(), configuration);
+    executorService = Executors.newCachedThreadPool();
+  }
+
+  @Override public Iterator<CarbonRowBatch>[] execute() {
+    int batchSize = CarbonProperties.getInstance().getBatchSize();
+    List<CarbonIterator<Object[]>>[] readerIterators = partitionInputReaderIterators();
+    Iterator<CarbonRowBatch>[] outIterators = new Iterator[readerIterators.length];
+    for (int i = 0; i < outIterators.length; i++) {
+      outIterators[i] =
+          new InputProcessorIterator(readerIterators[i], rowParser, batchSize,
+              configuration.isPreFetch(), executorService, rowCounter);
+    }
+    return outIterators;
+  }
+
+  /**
+   * Partition input iterators equally as per the number of threads.
+   * @return
+   */
+  private List<CarbonIterator<Object[]>>[] partitionInputReaderIterators() {
+    // Get the number of cores configured in property.
+    int numberOfCores = CarbonProperties.getInstance().getNumberOfCores();
+    // Get the minimum of number of cores and iterators size to get the number of parallel threads
+    // to be launched.
+    int parallelThreadNumber = Math.min(inputIterators.length, numberOfCores);
+
+    List<CarbonIterator<Object[]>>[] iterators = new List[parallelThreadNumber];
+    for (int i = 0; i < parallelThreadNumber; i++) {
+      iterators[i] = new ArrayList<>();
+    }
+    // Equally partition the iterators as per number of threads
+    for (int i = 0; i < inputIterators.length; i++) {
+      iterators[i % parallelThreadNumber].add(inputIterators[i]);
+    }
+    return iterators;
+  }
+
+  @Override protected CarbonRow processRow(CarbonRow row) {
+    return null;
+  }
+
+  @Override public void close() {
+    if (!closed) {
+      super.close();
+      executorService.shutdown();
+      for (CarbonIterator inputIterator : inputIterators) {
+        inputIterator.close();
+      }
+    }
+  }
+
+  @Override protected String getStepName() {
+    return "Input Processor";
+  }
+
+  /**
+   * This iterator wraps the list of iterators and it starts iterating the each
+   * iterator of the list one by one. It also parse the data while iterating it.
+   */
+  private static class InputProcessorIterator extends CarbonIterator<CarbonRowBatch> {
+
+    private List<CarbonIterator<Object[]>> inputIterators;
+
+    private CarbonIterator<Object[]> currentIterator;
+
+    private int counter;
+
+    private int batchSize;
+
+    private RowParser rowParser;
+
+    private Future<CarbonRowBatch> future;
+
+    private ExecutorService executorService;
+
+    private boolean nextBatch;
+
+    private boolean firstTime;
+
+    private boolean preFetch;
+
+    private AtomicLong rowCounter;
+
+    public InputProcessorIterator(List<CarbonIterator<Object[]>> inputIterators,
+        RowParser rowParser, int batchSize, boolean preFetch, ExecutorService executorService,
+        AtomicLong rowCounter) {
+      this.inputIterators = inputIterators;
+      this.batchSize = batchSize;
+      this.rowParser = rowParser;
+      this.counter = 0;
+      // Get the first iterator from the list.
+      currentIterator = inputIterators.get(counter++);
+      this.executorService = executorService;
+      this.rowCounter = rowCounter;
+      this.preFetch = preFetch;
+      this.nextBatch = false;
+      this.firstTime = true;
+    }
+
+    @Override
+    public boolean hasNext() {
+      return nextBatch || internalHasNext();
+    }
+
+    private boolean internalHasNext() {
+      if (firstTime) {
+        firstTime = false;
+        currentIterator.initialize();
+      }
+      boolean hasNext = currentIterator.hasNext();
+      // If iterator is finished then check for next iterator.
+      if (!hasNext) {
+        currentIterator.close();
+        // Check next iterator is available in the list.
+        if (counter < inputIterators.size()) {
+          // Get the next iterator from the list.
+          currentIterator = inputIterators.get(counter++);
+          // Initialize the new iterator
+          currentIterator.initialize();
+          hasNext = internalHasNext();
+        }
+      }
+      return hasNext;
+    }
+
+    @Override
+    public CarbonRowBatch next() {
+      if (preFetch) {
+        return getCarbonRowBatchWithPreFetch();
+      } else {
+        return getBatch();
+      }
+    }
+
+    private CarbonRowBatch getCarbonRowBatchWithPreFetch() {
+      CarbonRowBatch result = null;
+      if (future == null) {
+        future = getCarbonRowBatch();
+      }
+      try {
+        result = future.get();
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e);
+      } catch (ExecutionException e) {
+        throw new RuntimeException(e);
+      }
+      nextBatch = false;
+      if (hasNext()) {
+        nextBatch = true;
+        future = getCarbonRowBatch();
+      }
+
+      return result;
+    }
+
+    private Future<CarbonRowBatch> getCarbonRowBatch() {
+      return executorService.submit(new Callable<CarbonRowBatch>() {
+        @Override public CarbonRowBatch call() throws Exception {
+          return getBatch();
+
+        }
+      });
+    }
+
+    private CarbonRowBatch getBatch() {
+      // Create batch and fill it.
+      CarbonRowBatch carbonRowBatch = new CarbonRowBatch(batchSize);
+      int count = 0;
+      while (internalHasNext() && count < batchSize) {
+        carbonRowBatch.addRow(new CarbonRow(rowParser.parseRow(currentIterator.next())));
+        count++;
+      }
+      rowCounter.getAndAdd(carbonRowBatch.getSize());
+      return carbonRowBatch;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/loading/steps/SortProcessorStepImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/steps/SortProcessorStepImpl.java b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/SortProcessorStepImpl.java
new file mode 100644
index 0000000..856d68c
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/steps/SortProcessorStepImpl.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.processing.loading.steps;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.processing.loading.AbstractDataLoadProcessorStep;
+import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration;
+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.Sorter;
+import org.apache.carbondata.processing.loading.sort.SorterFactory;
+import org.apache.carbondata.processing.sort.sortdata.SortParameters;
+
+/**
+ * It sorts the data and write them to intermediate temp files. These files will be further read
+ * by next step for writing to carbondata files.
+ */
+public class SortProcessorStepImpl extends AbstractDataLoadProcessorStep {
+
+  private Sorter sorter;
+
+  public SortProcessorStepImpl(CarbonDataLoadConfiguration configuration,
+      AbstractDataLoadProcessorStep child) {
+    super(configuration, child);
+  }
+
+  @Override
+  public DataField[] getOutput() {
+    return child.getOutput();
+  }
+
+  @Override
+  public void initialize() throws IOException {
+    super.initialize();
+    child.initialize();
+    SortParameters sortParameters = SortParameters.createSortParameters(configuration);
+    sorter = SorterFactory.createSorter(configuration, rowCounter);
+    sorter.initialize(sortParameters);
+  }
+
+  @Override
+  public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
+    final Iterator<CarbonRowBatch>[] iterators = child.execute();
+    return sorter.sort(iterators);
+  }
+
+  @Override
+  protected CarbonRow processRow(CarbonRow row) {
+    return null;
+  }
+
+  @Override
+  public void close() {
+    if (!closed) {
+      super.close();
+      if (sorter != null) {
+        sorter.close();
+      }
+    }
+  }
+
+  @Override protected String getStepName() {
+    return "Sort Processor";
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 f76c66f..aa77fb6 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
@@ -23,7 +23,7 @@ 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.model.CarbonLoadModel;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
 import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 838e5be..8631e1d 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
@@ -52,7 +52,7 @@ 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.core.writer.CarbonDeleteDeltaWriterImpl;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 
 /**
  * utility class for load merging.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 095e5a3..edffae9 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
@@ -33,15 +33,15 @@ import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
-import org.apache.carbondata.processing.sortandgroupby.exception.CarbonSortKeyAndGroupByException;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortDataRows;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortIntermediateFileMerger;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.SortParameters;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
+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.store.CarbonFactDataHandlerModel;
 import org.apache.carbondata.processing.store.CarbonFactHandler;
 import org.apache.carbondata.processing.store.CarbonFactHandlerFactory;
-import org.apache.carbondata.processing.store.SingleThreadFinalSortFilesMerger;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 import org.apache.spark.sql.types.Decimal;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/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 ef53163..f82f365 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
@@ -34,8 +34,8 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.scan.result.iterator.RawResultIterator;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
 import org.apache.carbondata.core.util.ByteUtil;
-import org.apache.carbondata.processing.merger.exeception.SliceMergerException;
-import org.apache.carbondata.processing.model.CarbonLoadModel;
+import org.apache.carbondata.processing.exception.SliceMergerException;
+import org.apache.carbondata.processing.loading.model.CarbonLoadModel;
 import org.apache.carbondata.processing.store.CarbonFactDataHandlerColumnar;
 import org.apache.carbondata.processing.store.CarbonFactDataHandlerModel;
 import org.apache.carbondata.processing.store.CarbonFactHandler;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/merger/exeception/SliceMergerException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/exeception/SliceMergerException.java b/processing/src/main/java/org/apache/carbondata/processing/merger/exeception/SliceMergerException.java
deleted file mode 100644
index 3ae3604..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/exeception/SliceMergerException.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.merger.exeception;
-
-import java.util.Locale;
-
-public class SliceMergerException 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 SliceMergerException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public SliceMergerException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/model/CarbonDataLoadSchema.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonDataLoadSchema.java b/processing/src/main/java/org/apache/carbondata/processing/model/CarbonDataLoadSchema.java
deleted file mode 100644
index cbdd7b4..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonDataLoadSchema.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.processing.model;
-
-import java.io.Serializable;
-
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-
-/**
- * Wrapper Data Load Schema object which will be used to
- * support relation while data loading
- */
-public class CarbonDataLoadSchema implements Serializable {
-
-  /**
-   * default serializer
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * CarbonTable info
-   */
-  private CarbonTable carbonTable;
-
-  /**
-   * CarbonDataLoadSchema constructor which takes CarbonTable
-   *
-   * @param carbonTable
-   */
-  public CarbonDataLoadSchema(CarbonTable carbonTable) {
-    this.carbonTable = carbonTable;
-  }
-
-  /**
-   * get carbontable
-   *
-   * @return carbonTable
-   */
-  public CarbonTable getCarbonTable() {
-    return carbonTable;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
deleted file mode 100644
index be2c8a5..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
+++ /dev/null
@@ -1,764 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.model;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.List;
-
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
-
-public class CarbonLoadModel implements Serializable {
-
-  private static final long serialVersionUID = 6580168429197697465L;
-
-  private String databaseName;
-
-  private String tableName;
-
-  private String factFilePath;
-
-  private String colDictFilePath;
-
-  private String partitionId;
-
-  private CarbonDataLoadSchema carbonDataLoadSchema;
-
-  private boolean aggLoadRequest;
-
-  private String storePath;
-
-  private boolean isRetentionRequest;
-
-  private String csvHeader;
-  private String[] csvHeaderColumns;
-  private String csvDelimiter;
-  private String complexDelimiterLevel1;
-  private String complexDelimiterLevel2;
-
-  private boolean isDirectLoad;
-  private List<LoadMetadataDetails> loadMetadataDetails;
-  private transient SegmentUpdateStatusManager segmentUpdateStatusManager;
-
-  private String blocksID;
-
-  /**
-   * Map from carbon dimension to pre defined dict file path
-   */
-  private HashMap<CarbonDimension, String> predefDictMap;
-
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-  /**
-   * new load start time
-   */
-  private long factTimeStamp;
-  /**
-   * load Id
-   */
-  private String segmentId;
-
-  private String allDictPath;
-
-  /**
-   * escape Char
-   */
-  private String escapeChar;
-
-  /**
-   * quote Char
-   */
-  private String quoteChar;
-
-  /**
-   * comment Char
-   */
-  private String commentChar;
-
-  private String dateFormat;
-
-  private String defaultTimestampFormat;
-
-  private String defaultDateFormat;
-
-  /**
-   * defines the string that should be treated as null while loadind data
-   */
-  private String serializationNullFormat;
-
-  /**
-   * defines the string to specify whether the bad record logger should be enabled or not
-   */
-  private String badRecordsLoggerEnable;
-
-  /**
-   * defines the option to specify the bad record logger action
-   */
-  private String badRecordsAction;
-
-  /**
-   * Max number of columns that needs to be parsed by univocity parser
-   */
-  private String maxColumns;
-
-  /**
-   * defines the string to specify whether empty data is good or bad
-   */
-  private String isEmptyDataBadRecord;
-
-  /**
-   * Use one pass to generate dictionary
-   */
-  private boolean useOnePass;
-
-  /**
-   * dictionary server host
-   */
-  private String dictionaryServerHost;
-
-  /**
-   * dictionary sever port
-   */
-  private int dictionaryServerPort;
-
-  /**
-   * Pre fetch data from csv reader
-   */
-  private boolean preFetch;
-
-  /**
-   * Batch sort should be enabled or not
-   */
-  private String sortScope;
-
-  /**
-   * Batch sort size in mb.
-   */
-  private String batchSortSizeInMb;
-  /**
-   * bad record location
-   */
-  private String badRecordsLocation;
-
-  /**
-   * Number of partitions in global sort.
-   */
-  private String globalSortPartitions;
-
-  /**
-   * get escape char
-   *
-   * @return
-   */
-  public String getEscapeChar() {
-    return escapeChar;
-  }
-
-  /**
-   * set escape char
-   *
-   * @param escapeChar
-   */
-  public void setEscapeChar(String escapeChar) {
-    this.escapeChar = escapeChar;
-  }
-
-  public String getCsvDelimiter() {
-    return csvDelimiter;
-  }
-
-  public void setCsvDelimiter(String csvDelimiter) {
-    this.csvDelimiter = csvDelimiter;
-  }
-
-  public String getComplexDelimiterLevel1() {
-    return complexDelimiterLevel1;
-  }
-
-  public void setComplexDelimiterLevel1(String complexDelimiterLevel1) {
-    this.complexDelimiterLevel1 = complexDelimiterLevel1;
-  }
-
-  public String getComplexDelimiterLevel2() {
-    return complexDelimiterLevel2;
-  }
-
-  public void setComplexDelimiterLevel2(String complexDelimiterLevel2) {
-    this.complexDelimiterLevel2 = complexDelimiterLevel2;
-  }
-
-  public boolean isDirectLoad() {
-    return isDirectLoad;
-  }
-
-  public void setDirectLoad(boolean isDirectLoad) {
-    this.isDirectLoad = isDirectLoad;
-  }
-
-  public String getAllDictPath() {
-    return allDictPath;
-  }
-
-  public void setAllDictPath(String allDictPath) {
-    this.allDictPath = allDictPath;
-  }
-
-  public String getCsvHeader() {
-    return csvHeader;
-  }
-
-  public void setCsvHeader(String csvHeader) {
-    this.csvHeader = csvHeader;
-  }
-
-  public String[] getCsvHeaderColumns() {
-    return csvHeaderColumns;
-  }
-
-  public void setCsvHeaderColumns(String[] csvHeaderColumns) {
-    this.csvHeaderColumns = csvHeaderColumns;
-  }
-
-  public void initPredefDictMap() {
-    predefDictMap = new HashMap<>();
-  }
-
-  public String getPredefDictFilePath(CarbonDimension dimension) {
-    return predefDictMap.get(dimension);
-  }
-
-  public void setPredefDictMap(CarbonDimension dimension, String predefDictFilePath) {
-    this.predefDictMap.put(dimension, predefDictFilePath);
-  }
-
-  /**
-   * @return carbon dataload schema
-   */
-  public CarbonDataLoadSchema getCarbonDataLoadSchema() {
-    return carbonDataLoadSchema;
-  }
-
-  /**
-   * @param carbonDataLoadSchema
-   */
-  public void setCarbonDataLoadSchema(CarbonDataLoadSchema carbonDataLoadSchema) {
-    this.carbonDataLoadSchema = carbonDataLoadSchema;
-  }
-
-  /**
-   * @return the databaseName
-   */
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  /**
-   * @param databaseName the databaseName to set
-   */
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * @return the tableName
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * @param tableName the tableName to set
-   */
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  /**
-   * @return the factFilePath
-   */
-  public String getFactFilePath() {
-    return factFilePath;
-  }
-
-  /**
-   * @param factFilePath the factFilePath to set
-   */
-  public void setFactFilePath(String factFilePath) {
-    this.factFilePath = factFilePath;
-  }
-
-  /**
-   * @return external column dictionary file path
-   */
-  public String getColDictFilePath() {
-    return colDictFilePath;
-  }
-
-  /**
-   * set external column dictionary file path
-   *
-   * @param colDictFilePath
-   */
-  public void setColDictFilePath(String colDictFilePath) {
-    this.colDictFilePath = colDictFilePath;
-  }
-
-  /**
-   * get copy with partition
-   *
-   * @param uniqueId
-   * @return
-   */
-  public CarbonLoadModel getCopyWithPartition(String uniqueId) {
-    CarbonLoadModel copy = new CarbonLoadModel();
-    copy.tableName = tableName;
-    copy.factFilePath = factFilePath + '/' + uniqueId;
-    copy.databaseName = databaseName;
-    copy.partitionId = uniqueId;
-    copy.aggLoadRequest = aggLoadRequest;
-    copy.loadMetadataDetails = loadMetadataDetails;
-    copy.isRetentionRequest = isRetentionRequest;
-    copy.complexDelimiterLevel1 = complexDelimiterLevel1;
-    copy.complexDelimiterLevel2 = complexDelimiterLevel2;
-    copy.carbonDataLoadSchema = carbonDataLoadSchema;
-    copy.blocksID = blocksID;
-    copy.taskNo = taskNo;
-    copy.factTimeStamp = factTimeStamp;
-    copy.segmentId = segmentId;
-    copy.serializationNullFormat = serializationNullFormat;
-    copy.badRecordsLoggerEnable = badRecordsLoggerEnable;
-    copy.badRecordsAction = badRecordsAction;
-    copy.escapeChar = escapeChar;
-    copy.quoteChar = quoteChar;
-    copy.commentChar = commentChar;
-    copy.dateFormat = dateFormat;
-    copy.defaultTimestampFormat = defaultTimestampFormat;
-    copy.maxColumns = maxColumns;
-    copy.storePath = storePath;
-    copy.useOnePass = useOnePass;
-    copy.dictionaryServerHost = dictionaryServerHost;
-    copy.dictionaryServerPort = dictionaryServerPort;
-    copy.preFetch = preFetch;
-    copy.isEmptyDataBadRecord = isEmptyDataBadRecord;
-    copy.sortScope = sortScope;
-    copy.batchSortSizeInMb = batchSortSizeInMb;
-    copy.badRecordsLocation = badRecordsLocation;
-    return copy;
-  }
-
-  /**
-   * Get copy with taskNo.
-   * Broadcast value is shared in process, so we need to copy it to make sure the value in each
-   * task independently.
-   *
-   * @return
-   */
-  public CarbonLoadModel getCopyWithTaskNo(String taskNo) {
-    CarbonLoadModel copy = new CarbonLoadModel();
-    copy.tableName = tableName;
-    copy.factFilePath = factFilePath;
-    copy.databaseName = databaseName;
-    copy.partitionId = partitionId;
-    copy.aggLoadRequest = aggLoadRequest;
-    copy.loadMetadataDetails = loadMetadataDetails;
-    copy.isRetentionRequest = isRetentionRequest;
-    copy.csvHeader = csvHeader;
-    copy.csvHeaderColumns = csvHeaderColumns;
-    copy.isDirectLoad = isDirectLoad;
-    copy.csvDelimiter = csvDelimiter;
-    copy.complexDelimiterLevel1 = complexDelimiterLevel1;
-    copy.complexDelimiterLevel2 = complexDelimiterLevel2;
-    copy.carbonDataLoadSchema = carbonDataLoadSchema;
-    copy.blocksID = blocksID;
-    copy.taskNo = taskNo;
-    copy.factTimeStamp = factTimeStamp;
-    copy.segmentId = segmentId;
-    copy.serializationNullFormat = serializationNullFormat;
-    copy.badRecordsLoggerEnable = badRecordsLoggerEnable;
-    copy.badRecordsAction = badRecordsAction;
-    copy.escapeChar = escapeChar;
-    copy.quoteChar = quoteChar;
-    copy.commentChar = commentChar;
-    copy.dateFormat = dateFormat;
-    copy.defaultTimestampFormat = defaultTimestampFormat;
-    copy.maxColumns = maxColumns;
-    copy.storePath = storePath;
-    copy.useOnePass = useOnePass;
-    copy.dictionaryServerHost = dictionaryServerHost;
-    copy.dictionaryServerPort = dictionaryServerPort;
-    copy.preFetch = preFetch;
-    copy.isEmptyDataBadRecord = isEmptyDataBadRecord;
-    copy.sortScope = sortScope;
-    copy.batchSortSizeInMb = batchSortSizeInMb;
-    return copy;
-  }
-
-  /**
-   * get CarbonLoadModel with partition
-   *
-   * @param uniqueId
-   * @param filesForPartition
-   * @param header
-   * @param delimiter
-   * @return
-   */
-  public CarbonLoadModel getCopyWithPartition(String uniqueId, List<String> filesForPartition,
-      String header, String delimiter) {
-    CarbonLoadModel copyObj = new CarbonLoadModel();
-    copyObj.tableName = tableName;
-    copyObj.factFilePath = null;
-    copyObj.databaseName = databaseName;
-    copyObj.partitionId = uniqueId;
-    copyObj.aggLoadRequest = aggLoadRequest;
-    copyObj.loadMetadataDetails = loadMetadataDetails;
-    copyObj.isRetentionRequest = isRetentionRequest;
-    copyObj.carbonDataLoadSchema = carbonDataLoadSchema;
-    copyObj.csvHeader = header;
-    copyObj.csvHeaderColumns = csvHeaderColumns;
-    copyObj.isDirectLoad = true;
-    copyObj.csvDelimiter = delimiter;
-    copyObj.complexDelimiterLevel1 = complexDelimiterLevel1;
-    copyObj.complexDelimiterLevel2 = complexDelimiterLevel2;
-    copyObj.blocksID = blocksID;
-    copyObj.taskNo = taskNo;
-    copyObj.factTimeStamp = factTimeStamp;
-    copyObj.segmentId = segmentId;
-    copyObj.serializationNullFormat = serializationNullFormat;
-    copyObj.badRecordsLoggerEnable = badRecordsLoggerEnable;
-    copyObj.badRecordsAction = badRecordsAction;
-    copyObj.escapeChar = escapeChar;
-    copyObj.quoteChar = quoteChar;
-    copyObj.commentChar = commentChar;
-    copyObj.dateFormat = dateFormat;
-    copyObj.defaultTimestampFormat = defaultTimestampFormat;
-    copyObj.maxColumns = maxColumns;
-    copyObj.storePath = storePath;
-    copyObj.useOnePass = useOnePass;
-    copyObj.dictionaryServerHost = dictionaryServerHost;
-    copyObj.dictionaryServerPort = dictionaryServerPort;
-    copyObj.preFetch = preFetch;
-    copyObj.isEmptyDataBadRecord = isEmptyDataBadRecord;
-    copyObj.sortScope = sortScope;
-    copyObj.batchSortSizeInMb = batchSortSizeInMb;
-    copyObj.badRecordsLocation = badRecordsLocation;
-    return copyObj;
-  }
-
-  /**
-   * @return the partitionId
-   */
-  public String getPartitionId() {
-    return partitionId;
-  }
-
-  /**
-   * @param partitionId the partitionId to set
-   */
-  public void setPartitionId(String partitionId) {
-    this.partitionId = partitionId;
-  }
-
-  /**
-   * @param storePath The storePath to set.
-   */
-  public void setStorePath(String storePath) {
-    this.storePath = storePath;
-  }
-
-  /**
-   * @return Returns the factStoreLocation.
-   */
-  public String getStorePath() {
-    return storePath;
-  }
-
-  /**
-   * isRetentionRequest
-   *
-   * @return
-   */
-  public boolean isRetentionRequest() {
-    return isRetentionRequest;
-  }
-
-  /**
-   * getLoadMetadataDetails.
-   *
-   * @return
-   */
-  public List<LoadMetadataDetails> getLoadMetadataDetails() {
-    return loadMetadataDetails;
-  }
-
-  /**
-   * setLoadMetadataDetails.
-   *
-   * @param loadMetadataDetails
-   */
-  public void setLoadMetadataDetails(List<LoadMetadataDetails> loadMetadataDetails) {
-    this.loadMetadataDetails = loadMetadataDetails;
-  }
-
-  /**
-   * getSegmentUpdateStatusManager
-   *
-   * @return
-   */
-  public SegmentUpdateStatusManager getSegmentUpdateStatusManager() {
-    return segmentUpdateStatusManager;
-  }
-
-  /**
-   * setSegmentUpdateStatusManager
-   *
-   * @param segmentUpdateStatusManager
-   */
-  public void setSegmentUpdateStatusManager(SegmentUpdateStatusManager segmentUpdateStatusManager) {
-    this.segmentUpdateStatusManager = segmentUpdateStatusManager;
-  }
-
-  /**
-   * @return
-   */
-  public String getTaskNo() {
-    return taskNo;
-  }
-
-  /**
-   * @param taskNo
-   */
-  public void setTaskNo(String taskNo) {
-    this.taskNo = taskNo;
-  }
-
-  /**
-   * @return
-   */
-  public long getFactTimeStamp() {
-    return factTimeStamp;
-  }
-
-  /**
-   * @param factTimeStamp
-   */
-  public void setFactTimeStamp(long factTimeStamp) {
-    this.factTimeStamp = factTimeStamp;
-  }
-
-  public String[] getDelimiters() {
-    return new String[] { complexDelimiterLevel1, complexDelimiterLevel2 };
-  }
-
-  /**
-   * @return load Id
-   */
-  public String getSegmentId() {
-    return segmentId;
-  }
-
-  /**
-   * @param segmentId
-   */
-  public void setSegmentId(String segmentId) {
-    this.segmentId = segmentId;
-  }
-
-  /**
-   * the method returns the value to be treated as null while data load
-   *
-   * @return
-   */
-  public String getSerializationNullFormat() {
-    return serializationNullFormat;
-  }
-
-  /**
-   * the method sets the value to be treated as null while data load
-   *
-   * @param serializationNullFormat
-   */
-  public void setSerializationNullFormat(String serializationNullFormat) {
-    this.serializationNullFormat = serializationNullFormat;
-  }
-
-  /**
-   * returns the string to enable bad record logger
-   *
-   * @return
-   */
-  public String getBadRecordsLoggerEnable() {
-    return badRecordsLoggerEnable;
-  }
-
-  /**
-   * method sets the string to specify whether to enable or dissable the badrecord logger.
-   *
-   * @param badRecordsLoggerEnable
-   */
-  public void setBadRecordsLoggerEnable(String badRecordsLoggerEnable) {
-    this.badRecordsLoggerEnable = badRecordsLoggerEnable;
-  }
-
-  public String getQuoteChar() {
-    return quoteChar;
-  }
-
-  public void setQuoteChar(String quoteChar) {
-    this.quoteChar = quoteChar;
-  }
-
-  public String getCommentChar() {
-    return commentChar;
-  }
-
-  public void setCommentChar(String commentChar) {
-    this.commentChar = commentChar;
-  }
-
-  public String getDateFormat() {
-    return dateFormat;
-  }
-
-  public void setDateFormat(String dateFormat) {
-    this.dateFormat = dateFormat;
-  }
-
-  public String getDefaultTimestampFormat() {
-    return defaultTimestampFormat;
-  }
-
-  public void setDefaultTimestampFormat(String defaultTimestampFormat) {
-    this.defaultTimestampFormat = defaultTimestampFormat;
-  }
-
-  /**
-   * @return
-   */
-  public String getMaxColumns() {
-    return maxColumns;
-  }
-
-  /**
-   * @param maxColumns
-   */
-  public void setMaxColumns(String maxColumns) {
-    this.maxColumns = maxColumns;
-  }
-
-  /**
-   * returns option to specify the bad record logger action
-   *
-   * @return
-   */
-  public String getBadRecordsAction() {
-    return badRecordsAction;
-  }
-
-  /**
-   * set option to specify the bad record logger action
-   *
-   * @param badRecordsAction
-   */
-  public void setBadRecordsAction(String badRecordsAction) {
-    this.badRecordsAction = badRecordsAction;
-  }
-
-  public boolean getUseOnePass() {
-    return useOnePass;
-  }
-
-  public void setUseOnePass(boolean useOnePass) {
-    this.useOnePass = useOnePass;
-  }
-
-  public int getDictionaryServerPort() {
-    return dictionaryServerPort;
-  }
-
-  public void setDictionaryServerPort(int dictionaryServerPort) {
-    this.dictionaryServerPort = dictionaryServerPort;
-  }
-
-  public String getDictionaryServerHost() {
-    return dictionaryServerHost;
-  }
-
-  public void setDictionaryServerHost(String dictionaryServerHost) {
-    this.dictionaryServerHost = dictionaryServerHost;
-  }
-
-  public boolean isPreFetch() {
-    return preFetch;
-  }
-
-  public void setPreFetch(boolean preFetch) {
-    this.preFetch = preFetch;
-  }
-
-  public String getDefaultDateFormat() {
-    return defaultDateFormat;
-  }
-
-  public void setDefaultDateFormat(String defaultDateFormat) {
-    this.defaultDateFormat = defaultDateFormat;
-  }
-
-  public String getIsEmptyDataBadRecord() {
-    return isEmptyDataBadRecord;
-  }
-
-  public void setIsEmptyDataBadRecord(String isEmptyDataBadRecord) {
-    this.isEmptyDataBadRecord = isEmptyDataBadRecord;
-  }
-
-  public String getSortScope() {
-    return sortScope;
-  }
-
-  public void setSortScope(String sortScope) {
-    this.sortScope = sortScope;
-  }
-
-  public String getBatchSortSizeInMb() {
-    return batchSortSizeInMb;
-  }
-
-  public void setBatchSortSizeInMb(String batchSortSizeInMb) {
-    this.batchSortSizeInMb = batchSortSizeInMb;
-  }
-
-  public String getGlobalSortPartitions() {
-    return globalSortPartitions;
-  }
-
-  public void setGlobalSortPartitions(String globalSortPartitions) {
-    this.globalSortPartitions = globalSortPartitions;
-  }
-
-  public String getBadRecordsLocation() {
-    return badRecordsLocation;
-  }
-
-  public void setBadRecordsLocation(String badRecordsLocation) {
-    this.badRecordsLocation = badRecordsLocation;
-  }
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/349c59c7/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java
deleted file mode 100644
index 9e0aa02..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/AbstractDataLoadProcessorStep.java
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.newflow;
-
-import java.io.IOException;
-import java.util.Iterator;
-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.datastore.row.CarbonRow;
-import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException;
-import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
-
-/**
- * This base abstract class for data loading.
- * It can do transformation jobs as per the implementation.
- *
- * Life cycle of this class is
- * First initialize() is called to initialize the step
- * then execute() is called to process the step logic and
- * then close() is called to close any resources if any opened in the step.
- */
-public abstract class AbstractDataLoadProcessorStep {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractDataLoadProcessorStep.class.getName());
-
-  protected CarbonDataLoadConfiguration configuration;
-
-  protected AbstractDataLoadProcessorStep child;
-
-  protected AtomicLong rowCounter;
-
-  protected boolean closed;
-
-  public AbstractDataLoadProcessorStep(CarbonDataLoadConfiguration configuration,
-      AbstractDataLoadProcessorStep child) {
-    this.configuration = configuration;
-    this.child = child;
-    this.rowCounter = new AtomicLong();
-    this.closed = false;
-  }
-
-  /**
-   * The output meta for this step. The data returns from this step is as per this meta.
-   *
-   */
-  public abstract DataField[] getOutput();
-
-  /**
-   * Initialization process for this step.
-   *
-   * @throws IOException
-   */
-  public void initialize() throws IOException {
-    if (LOGGER.isInfoEnabled()) {
-      // This thread prints the rows processed in each step for every 10 seconds.
-      new Thread() {
-        @Override public void run() {
-          while (!closed) {
-            try {
-              LOGGER.info("Rows processed in step " + getStepName() + " : " + rowCounter.get());
-              Thread.sleep(10000);
-            } catch (InterruptedException e) {
-              //ignore
-              LOGGER.error(e.getMessage());
-            }
-          }
-        }
-      }.start();
-    }
-  }
-
-  /**
-   * Tranform the data as per the implementation.
-   *
-   * @return Array of Iterator with data. It can be processed parallel if implementation class wants
-   * @throws CarbonDataLoadingException
-   */
-  public Iterator<CarbonRowBatch>[] execute() throws CarbonDataLoadingException {
-    Iterator<CarbonRowBatch>[] childIters = child.execute();
-    Iterator<CarbonRowBatch>[] iterators = new Iterator[childIters.length];
-    for (int i = 0; i < childIters.length; i++) {
-      iterators[i] = getIterator(childIters[i]);
-    }
-    return iterators;
-  }
-
-  /**
-   * Create the iterator using child iterator.
-   *
-   * @param childIter
-   * @return new iterator with step specific processing.
-   */
-  protected Iterator<CarbonRowBatch> getIterator(final Iterator<CarbonRowBatch> childIter) {
-    return new CarbonIterator<CarbonRowBatch>() {
-      @Override public boolean hasNext() {
-        return childIter.hasNext();
-      }
-
-      @Override public CarbonRowBatch next() {
-        return processRowBatch(childIter.next());
-      }
-    };
-  }
-
-  /**
-   * Process the batch of rows as per the step logic.
-   *
-   * @param rowBatch
-   * @return processed row.
-   */
-  protected CarbonRowBatch processRowBatch(CarbonRowBatch rowBatch) {
-    CarbonRowBatch newBatch = new CarbonRowBatch(rowBatch.getSize());
-    while (rowBatch.hasNext()) {
-      newBatch.addRow(processRow(rowBatch.next()));
-    }
-    return newBatch;
-  }
-
-  /**
-   * Process the row as per the step logic.
-   *
-   * @param row
-   * @return processed row.
-   */
-  protected abstract CarbonRow processRow(CarbonRow row);
-
-  /**
-   * Get the step name for logging purpose.
-   * @return Step name
-   */
-  protected abstract String getStepName();
-
-
-  /**
-   * Close all resources.This method is called after execute() is finished.
-   * It will be called in both success and failure cases.
-   */
-  public void close() {
-    if (!closed) {
-      closed = true;
-      LOGGER.info("Total rows processed in step " + this.getStepName() + ": " + rowCounter.get());
-      if (child != null) {
-        child.close();
-      }
-    }
-  }
-
-}