You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2016/12/29 08:13:26 UTC

[2/3] incubator-carbondata git commit: wip

wip

clean up unused declare

fix style

fix comment

fix style


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

Branch: refs/heads/master
Commit: 3fe69037e741b0fa71a1d2369eff3a8df6c4e723
Parents: 93f23cc
Author: jackylk <ja...@huawei.com>
Authored: Wed Dec 28 01:37:36 2016 +0800
Committer: chenliang613 <ch...@apache.org>
Committed: Thu Dec 29 16:11:07 2016 +0800

----------------------------------------------------------------------
 .../hadoop/test/util/StoreCreator.java          |  12 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  49 +---
 .../lcm/status/SegmentStatusManager.java        |   4 -
 .../api/dataloader/DataLoadModel.java           |  56 ----
 .../processing/api/dataloader/SchemaInfo.java   |  76 -----
 .../processing/csvload/DataGraphExecuter.java   |  19 +-
 .../processing/csvload/GraphExecutionUtil.java  |   5 +-
 .../processing/csvreaderstep/CsvInput.java      | 201 -------------
 .../processing/dataprocessor/queue/Queue.java   |  47 ---
 .../queue/impl/DataProcessorQueue.java          | 106 -------
 .../holder/DataProcessorRecordHolder.java       |  25 --
 .../processing/datatypes/ArrayDataType.java     |   8 -
 .../processing/datatypes/GenericDataType.java   |   5 -
 .../processing/datatypes/PrimitiveDataType.java |  11 -
 .../processing/datatypes/StructDataType.java    |   8 -
 .../graphgenerator/GraphGenerator.java          |  43 +--
 .../configuration/GraphConfigurationInfo.java   | 153 ----------
 .../processing/mdkeygen/MDKeyGenStepMeta.java   |  27 --
 .../processing/mdkeygen/file/FileManager.java   |   7 -
 .../merger/step/CarbonSliceMergerStepMeta.java  |  90 ------
 .../processing/model/CarbonLoadModel.java       |  18 --
 .../newflow/CarbonDataLoadConfiguration.java    |   8 -
 .../processing/newflow/DataField.java           |  11 -
 .../newflow/iterator/InputIterator.java         |  40 ---
 .../sort/impl/ParallelReadMergeSorterImpl.java  |   6 -
 .../impl/UnsafeParallelReadMergeSorterImpl.java |   6 -
 .../sort/unsafe/UnsafeCarbonRowPage.java        |  30 --
 .../newflow/sort/unsafe/UnsafeSortDataRows.java |  39 ---
 .../holder/UnsafeInmemoryMergeHolder.java       |   4 -
 .../UnsafeInMemoryIntermediateDataMerger.java   |   1 -
 .../UnsafeSingleThreadFinalSortFilesMerger.java |   5 -
 .../newflow/steps/InputProcessorStepImpl.java   |   5 -
 .../processing/schema/metadata/ColumnsInfo.java |  72 -----
 .../schema/metadata/HierarchiesInfo.java        |  77 -----
 .../sortdata/AbstractTempSortFileReader.java    | 141 ---------
 .../sortdata/CompressedTempSortFileReader.java  |  52 ----
 .../sortandgroupby/sortdata/SortParameters.java |   4 +-
 .../sortdata/TempSortFileReaderFactory.java     |  45 ---
 .../UnCompressedTempSortFileReader.java         |  50 ----
 .../sortdatastep/SortKeyStep.java               |   2 +-
 .../sortdatastep/SortKeyStepMeta.java           |  32 ---
 .../store/CarbonFactDataHandlerColumnar.java    |  33 ---
 .../store/CarbonFactDataHandlerModel.java       |  64 -----
 .../store/writer/AbstractFactDataWriter.java    |  15 -
 .../store/writer/CarbonDataWriterVo.java        |  21 --
 .../store/writer/CarbonFactDataWriter.java      |  12 -
 .../processing/store/writer/NodeHolder.java     |  70 -----
 .../CarbonCSVBasedDimSurrogateKeyGen.java       |  16 --
 .../csvbased/CarbonCSVBasedSeqGenData.java      |  35 ---
 .../csvbased/CarbonCSVBasedSeqGenMeta.java      | 270 -----------------
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |   3 -
 .../FileStoreSurrogateKeyGenForCSV.java         |  36 ---
 .../dbbased/CarbonDimSurrogateKeyGen.java       | 286 -------------------
 .../dbbased/HierarchyValueWriter.java           | 155 ----------
 .../dbbased/IntArrayWrapper.java                | 110 -------
 .../processing/util/CarbonSchemaParser.java     | 223 +--------------
 .../processing/util/RemoveDictionaryUtil.java   |   9 -
 .../carbondata/test/util/StoreCreator.java      |  12 +-
 58 files changed, 29 insertions(+), 2941 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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 929e404..4519e90 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
@@ -476,20 +476,10 @@ public class StoreCreator {
     model.setQuoteCharacter(dataProcessTaskStatus.getQuoteCharacter());
     model.setCommentCharacter(dataProcessTaskStatus.getCommentCharacter());
     model.setDateFormat(dataProcessTaskStatus.getDateFormat());
-    if (null != loadMetadataDetails && !loadMetadataDetails.isEmpty()) {
-      model.setLoadNames(
-          CarbonDataProcessorUtil.getLoadNameFromLoadMetaDataDetails(loadMetadataDetails));
-      model.setModificationOrDeletionTime(CarbonDataProcessorUtil
-          .getModificationOrDeletionTimesFromLoadMetadataDetails(loadMetadataDetails));
-    }
-    boolean hdfsReadMode =
-        dataProcessTaskStatus.getCsvFilePath() != null && dataProcessTaskStatus.getCsvFilePath().startsWith("hdfs:");
-    int allocate = null != dataProcessTaskStatus.getCsvFilePath() ? 1 : dataProcessTaskStatus.getFilesToProcess().size();
     String outputLocation = CarbonProperties.getInstance()
         .getProperty("store_output_location", "../carbon-store/system/carbon/etl");
     GraphGenerator generator =
-        new GraphGenerator(model, hdfsReadMode, partitionID, factStoreLocation,
-            allocate, schema, "0", outputLocation);
+        new GraphGenerator(model, partitionID, factStoreLocation, schema, "0", outputLocation);
     generator.generateGraph();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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 56ddce2..eca34a6 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
@@ -77,7 +77,6 @@ import org.apache.carbondata.processing.dataprocessor.IDataProcessStatus;
 import org.apache.carbondata.processing.graphgenerator.GraphGenerator;
 import org.apache.carbondata.processing.graphgenerator.GraphGeneratorException;
 import org.apache.carbondata.processing.model.CarbonLoadModel;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 import org.apache.carbondata.spark.merger.NodeBlockRelation;
 import org.apache.carbondata.spark.merger.NodeMultiBlockRelation;
 
@@ -89,25 +88,8 @@ public final class CarbonLoaderUtil {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(CarbonLoaderUtil.class.getName());
-  /**
-   * minimum no of blocklet required for distribution
-   */
-  private static int minBlockLetsReqForDistribution = 0;
-
-  static {
-    String property = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_BLOCKLETDISTRIBUTION_MIN_REQUIRED_SIZE);
-    try {
-      minBlockLetsReqForDistribution = Integer.parseInt(property);
-    } catch (NumberFormatException ne) {
-      LOGGER.info("Invalid configuration. Consisering the defaul");
-      minBlockLetsReqForDistribution =
-          CarbonCommonConstants.DEFAULT_CARBON_BLOCKLETDISTRIBUTION_MIN_REQUIRED_SIZE;
-    }
-  }
 
   private CarbonLoaderUtil() {
-
   }
 
   private static void generateGraph(IDataProcessStatus dataProcessTaskStatus, SchemaInfo info,
@@ -118,13 +100,6 @@ public final class CarbonLoaderUtil {
             || null != dataProcessTaskStatus.getFilesToProcess());
     model.setSchemaInfo(info);
     model.setTableName(dataProcessTaskStatus.getTableName());
-    List<LoadMetadataDetails> loadMetadataDetails = loadModel.getLoadMetadataDetails();
-    if (null != loadMetadataDetails && !loadMetadataDetails.isEmpty()) {
-      model.setLoadNames(
-          CarbonDataProcessorUtil.getLoadNameFromLoadMetaDataDetails(loadMetadataDetails));
-      model.setModificationOrDeletionTime(CarbonDataProcessorUtil
-          .getModificationOrDeletionTimesFromLoadMetadataDetails(loadMetadataDetails));
-    }
     model.setBlocksID(dataProcessTaskStatus.getBlocksID());
     model.setEscapeCharacter(dataProcessTaskStatus.getEscapeCharacter());
     model.setQuoteCharacter(dataProcessTaskStatus.getQuoteCharacter());
@@ -134,15 +109,9 @@ public final class CarbonLoaderUtil {
     model.setFactTimeStamp(loadModel.getFactTimeStamp());
     model.setMaxColumns(loadModel.getMaxColumns());
     model.setDateFormat(loadModel.getDateFormat());
-    boolean hdfsReadMode =
-        dataProcessTaskStatus.getCsvFilePath() != null
-                && dataProcessTaskStatus.getCsvFilePath().startsWith("hdfs:");
-    int allocate =
-            null != dataProcessTaskStatus.getCsvFilePath()
-                    ? 1 : dataProcessTaskStatus.getFilesToProcess().size();
-    GraphGenerator generator = new GraphGenerator(model, hdfsReadMode, loadModel.getPartitionId(),
-        loadModel.getStorePath(), allocate,
-        loadModel.getCarbonDataLoadSchema(), loadModel.getSegmentId(), outputLocation);
+    GraphGenerator generator = new GraphGenerator(model, loadModel.getPartitionId(),
+        loadModel.getStorePath(), loadModel.getCarbonDataLoadSchema(), loadModel.getSegmentId(),
+        outputLocation);
     generator.generateGraph();
   }
 
@@ -160,7 +129,6 @@ public final class CarbonLoaderUtil {
     CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation);
     CarbonProperties.getInstance()
         .addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS, storePath);
-    // CarbonProperties.getInstance().addProperty("store_output_location", outPutLoc);
     CarbonProperties.getInstance().addProperty("send.signal.load", "false");
 
     String fileNamePrefix = "";
@@ -542,17 +510,6 @@ public final class CarbonLoaderUtil {
   }
 
   /**
-   * This method will divide the blocks among the nodes as per the data locality
-   *
-   * @param blockInfos
-   * @return
-   */
-  public static Map<String, List<Distributable>> nodeBlockMapping(List<Distributable> blockInfos) {
-    // -1 if number of nodes has to be decided based on block location information
-    return nodeBlockMapping(blockInfos, -1);
-  }
-
-  /**
    * the method returns the number of required executors
    *
    * @param blockInfos

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java b/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
index 030ee0b..b8b86b6 100644
--- a/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
+++ b/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
@@ -582,10 +582,6 @@ public class SegmentStatusManager {
       return validSegments;
     }
 
-    public List<String> getUpadtedSegments() {
-      return validUpdatedSegments;
-    }
-
     public List<String> getInvalidSegments() {
       return invalidSegments;
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 5a7f14d..3971c97 100644
--- 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
@@ -35,20 +35,6 @@ public class DataLoadModel {
    */
   private boolean isCsvLoad;
 
-  /**
-   * Modified Dimension
-   */
-  private String[] modifiedDimesion;
-
-  /**
-   * loadNames separated by HASH_SPC_CHARACTER
-   */
-  private String loadNames;
-  /**
-   * modificationOrDeletionTime separated by HASH_SPC_CHARACTER
-   */
-  private String modificationOrDeletionTime;
-
   private String blocksID;
   /**
    * task id, each spark task has a unique id
@@ -113,48 +99,6 @@ public class DataLoadModel {
   }
 
   /**
-   * @return Returns the modifiedDimesion.
-   */
-  public String[] getModifiedDimesion() {
-    return modifiedDimesion;
-  }
-
-  /**
-   * @param modifiedDimesion The modifiedDimesion to set.
-   */
-  public void setModifiedDimesion(String[] modifiedDimesion) {
-    this.modifiedDimesion = modifiedDimesion;
-  }
-
-  /**
-   * return modificationOrDeletionTime separated by HASH_SPC_CHARACTER
-   */
-  public String getModificationOrDeletionTime() {
-    return modificationOrDeletionTime;
-  }
-
-  /**
-   * set modificationOrDeletionTime separated by HASH_SPC_CHARACTER
-   */
-  public void setModificationOrDeletionTime(String modificationOrDeletionTime) {
-    this.modificationOrDeletionTime = modificationOrDeletionTime;
-  }
-
-  /**
-   * return loadNames separated by HASH_SPC_CHARACTER
-   */
-  public String getLoadNames() {
-    return loadNames;
-  }
-
-  /**
-   * set loadNames separated by HASH_SPC_CHARACTER
-   */
-  public void setLoadNames(String loadNames) {
-    this.loadNames = loadNames;
-  }
-
-  /**
    * get block id
    *
    * @return

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index a00e913..4b0d8bf 100644
--- 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
@@ -27,26 +27,6 @@ public class SchemaInfo {
   private String databaseName;
 
   /**
-   * srcDriverName
-   */
-  private String srcDriverName;
-
-  /**
-   * srcConUrl
-   */
-  private String srcConUrl;
-
-  /**
-   * srcUserName
-   */
-  private String srcUserName;
-
-  /**
-   * srcPwd
-   */
-  private String srcPwd;
-
-  /**
    * tableName
    */
   private String tableName;
@@ -90,62 +70,6 @@ public class SchemaInfo {
     this.complexDelimiterLevel2 = complexDelimiterLevel2;
   }
 
-  /**
-   * @return Returns the srcDriverName.
-   */
-  public String getSrcDriverName() {
-    return srcDriverName;
-  }
-
-  /**
-   * @param srcDriverName The srcDriverName to set.
-   */
-  public void setSrcDriverName(String srcDriverName) {
-    this.srcDriverName = srcDriverName;
-  }
-
-  /**
-   * @return Returns the srcConUrl.
-   */
-  public String getSrcConUrl() {
-    return srcConUrl;
-  }
-
-  /**
-   * @param srcConUrl The srcConUrl to set.
-   */
-  public void setSrcConUrl(String srcConUrl) {
-    this.srcConUrl = srcConUrl;
-  }
-
-  /**
-   * @return Returns the srcUserName.
-   */
-  public String getSrcUserName() {
-    return srcUserName;
-  }
-
-  /**
-   * @param srcUserName The srcUserName to set.
-   */
-  public void setSrcUserName(String srcUserName) {
-    this.srcUserName = srcUserName;
-  }
-
-  /**
-   * @return Returns the srcPwd.
-   */
-  public String getSrcPwd() {
-    return srcPwd;
-  }
-
-  /**
-   * @param srcPwd The srcPwd to set.
-   */
-  public void setSrcPwd(String srcPwd) {
-    this.srcPwd = srcPwd;
-  }
-
   public String getTableName() {
     return tableName;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
index 4765277..3d35837 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
@@ -260,7 +260,7 @@ public class DataGraphExecuter {
 
           CarbonFile csvFileToRead = GraphExecutionUtil.getCsvFileToRead(model.getCsvFilePath());
           TextFileInputField[] inputFields = GraphExecutionUtil
-              .getTextInputFiles(csvFileToRead, measureColumns, builder, measuresInCSVFile, ",");
+              .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile, ",");
           stepMetaInterface.setInputFields(inputFields);
         } else if (model.isDirectLoad()) {
           String[] files = new String[model.getFilesToProcess().size()];
@@ -293,7 +293,7 @@ public class DataGraphExecuter {
             CarbonFile csvFile =
                 GraphExecutionUtil.getCsvFileToRead(model.getFilesToProcess().get(0));
             TextFileInputField[] inputFields = GraphExecutionUtil
-                .getTextInputFiles(csvFile, measureColumns, builder, measuresInCSVFile,
+                .getTextInputFiles(csvFile, builder, measuresInCSVFile,
                     model.getCsvDelimiter());
             ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
             ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
@@ -369,7 +369,7 @@ public class DataGraphExecuter {
         if (null != model.getCsvFilePath() && model.getRddIteratorKey() == null) {
           CarbonFile csvFileToRead = GraphExecutionUtil.getCsvFileToRead(model.getCsvFilePath());
           TextFileInputField[] inputFields = GraphExecutionUtil
-              .getTextInputFiles(csvFileToRead, measureColumns, builder, measuresInCSVFile, ",");
+              .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile, ",");
           ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
         } else if (model.isDirectLoad()) {
           if (null != model.getCsvHeader() && !model.getCsvHeader().isEmpty()) {
@@ -385,7 +385,7 @@ public class DataGraphExecuter {
             CarbonFile csvFileToRead =
                 GraphExecutionUtil.getCsvFileToRead(model.getFilesToProcess().get(0));
             TextFileInputField[] inputFields = GraphExecutionUtil
-                .getTextInputFiles(csvFileToRead, measureColumns, builder, measuresInCSVFile,
+                .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile,
                     model.getCsvDelimiter());
             ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
             ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
@@ -541,15 +541,4 @@ public class DataGraphExecuter {
     return GraphExecutionUtil.checkCSVAndRequestedTableColumns(dimFilePath, columnNames, delimiter);
   }
 
-  /**
-   * Interrupts all child threads run by kettle to execute the graph
-   */
-  public void interruptGraphExecution() {
-    LOGGER.error("Graph Execution is interrupted");
-    if (null != trans) {
-      trans.killAll();
-      LOGGER.info("Graph execution steps are killed.");
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
index 1dd9bdf..2148901 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
@@ -93,9 +93,8 @@ public final class GraphExecutionUtil {
    * @param measuresInCSVFile
    * @throws DataLoadingException
    */
-  public static TextFileInputField[] getTextInputFiles(CarbonFile csvFile,
-      List<String> measureColumns, StringBuilder builder, StringBuilder measuresInCSVFile,
-      String delimiter) throws DataLoadingException {
+  public static TextFileInputField[] getTextInputFiles(CarbonFile csvFile, StringBuilder builder,
+      StringBuilder measuresInCSVFile, String delimiter) throws DataLoadingException {
     DataInputStream fileReader = null;
     BufferedReader bufferedReader = null;
     String readLine = null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
index c64c504..11acd56 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
@@ -41,7 +41,6 @@ import org.apache.carbondata.processing.graphgenerator.GraphGenerator;
 import org.apache.commons.lang3.StringUtils;
 import org.pentaho.di.core.Const;
 import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.logging.LogChannelInterface;
 import org.pentaho.di.core.row.RowMeta;
 import org.pentaho.di.core.row.ValueMetaInterface;
 import org.pentaho.di.i18n.BaseMessages;
@@ -81,7 +80,6 @@ public class CsvInput extends BaseStep implements StepInterface {
    * resultArray
    */
   private Future[] resultArray;
-  private boolean isTerminated;
   private List<List<BlockDetails>> threadBlockList = new ArrayList<>();
 
   private ExecutorService exec;
@@ -97,205 +95,6 @@ public class CsvInput extends BaseStep implements StepInterface {
     LOGGER.info("** Using csv file **");
   }
 
-  /**
-   * This method is borrowed from TextFileInput
-   *
-   * @param log
-   * @param line
-   * @param delimiter
-   * @param enclosure
-   * @param escapeCharacter
-   * @return
-   * @throws KettleException
-   */
-  public static final String[] guessStringsFromLine(LogChannelInterface log, String line,
-      String delimiter, String enclosure, String escapeCharacter) throws KettleException {
-    List<String> strings = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-    String pol; // piece of line
-
-    try {
-      if (line == null) {
-        return null;
-      }
-
-      // Split string in pieces, only for CSV!
-      int pos = 0;
-      int length = line.length();
-      boolean dencl = false;
-
-      int lenEncl = (enclosure == null ? 0 : enclosure.length());
-      int lenEsc = (escapeCharacter == null ? 0 : escapeCharacter.length());
-
-      while (pos < length) {
-        int from = pos;
-        int next;
-
-        boolean enclFound;
-        boolean containsEscapedEnclosures = false;
-        boolean containsEscapedSeparators = false;
-
-        // Is the field beginning with an enclosure?
-        // "aa;aa";123;"aaa-aaa";000;...
-        if (lenEncl > 0 && line.substring(from, from + lenEncl).equalsIgnoreCase(enclosure)) {
-          if (log.isRowLevel()) {
-            log.logRowlevel(BaseMessages.getString(PKG, "CsvInput.Log.ConvertLineToRowTitle"),
-                BaseMessages.getString(PKG, "CsvInput.Log.ConvertLineToRow",
-                    line.substring(from, from + lenEncl))); //$NON-NLS-1$ //$NON-NLS-2$
-          }
-          enclFound = true;
-          int p = from + lenEncl;
-
-          boolean isEnclosure =
-              lenEncl > 0 && p + lenEncl < length && line.substring(p, p + lenEncl)
-                  .equalsIgnoreCase(enclosure);
-          boolean isEscape = lenEsc > 0 && p + lenEsc < length && line.substring(p, p + lenEsc)
-              .equalsIgnoreCase(escapeCharacter);
-
-          boolean enclosureAfter = false;
-
-          // Is it really an enclosure? See if it's not repeated twice or escaped!
-          if ((isEnclosure || isEscape) && p < length - 1) {
-            String strnext = line.substring(p + lenEncl, p + 2 * lenEncl);
-            if (strnext.equalsIgnoreCase(enclosure)) {
-              p++;
-              enclosureAfter = true;
-              dencl = true;
-
-              // Remember to replace them later on!
-              if (isEscape) {
-                containsEscapedEnclosures = true;
-              }
-            }
-          }
-
-          // Look for a closing enclosure!
-          while ((!isEnclosure || enclosureAfter) && p < line.length()) {
-            p++;
-            enclosureAfter = false;
-            isEnclosure = lenEncl > 0 && p + lenEncl < length && line.substring(p, p + lenEncl)
-                .equals(enclosure);
-            isEscape = lenEsc > 0 && p + lenEsc < length && line.substring(p, p + lenEsc)
-                .equals(escapeCharacter);
-
-            // Is it really an enclosure? See if it's not repeated twice or escaped!
-            if ((isEnclosure || isEscape) && p < length - 1) // Is
-            {
-              String strnext = line.substring(p + lenEncl, p + 2 * lenEncl);
-              if (strnext.equals(enclosure)) {
-                p++;
-                enclosureAfter = true;
-                dencl = true;
-
-                // Remember to replace them later on!
-                if (isEscape) {
-                  containsEscapedEnclosures = true; // remember
-                }
-              }
-            }
-          }
-
-          if (p >= length) {
-            next = p;
-          } else {
-            next = p + lenEncl;
-          }
-
-          if (log.isRowLevel()) {
-            log.logRowlevel(BaseMessages.getString(PKG, "CsvInput.Log.ConvertLineToRowTitle"),
-                BaseMessages.getString(PKG, "CsvInput.Log.EndOfEnclosure",
-                    "" + p)); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
-          }
-        } else {
-          enclFound = false;
-          boolean found = false;
-          int startpoint = from;
-          do {
-            next = line.indexOf(delimiter, startpoint);
-
-            // See if this position is preceded by an escape character.
-            if (lenEsc > 0 && next - lenEsc > 0) {
-              String before = line.substring(next - lenEsc, next);
-
-              if (escapeCharacter != null && escapeCharacter.equals(before)) {
-                // take the next separator, this one is escaped...
-                startpoint = next + 1;
-                containsEscapedSeparators = true;
-              } else {
-                found = true;
-              }
-            } else {
-              found = true;
-            }
-          } while (!found && next >= 0);
-        }
-        if (next == -1) {
-          next = length;
-        }
-
-        if (enclFound) {
-          pol = line.substring(from + lenEncl, next - lenEncl);
-          if (log.isRowLevel()) {
-            log.logRowlevel(BaseMessages.getString(PKG, "CsvInput.Log.ConvertLineToRowTitle"),
-                BaseMessages.getString(PKG, "CsvInput.Log.EnclosureFieldFound",
-                    "" + pol)); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
-          }
-        } else {
-          pol = line.substring(from, next);
-          if (log.isRowLevel()) {
-            log.logRowlevel(BaseMessages.getString(PKG, "CsvInput.Log.ConvertLineToRowTitle"),
-                BaseMessages.getString(PKG, "CsvInput.Log.NormalFieldFound",
-                    "" + pol)); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
-          }
-        }
-
-        if (dencl) {
-          StringBuilder sbpol = new StringBuilder(pol);
-          int idx = sbpol.indexOf(enclosure + enclosure);
-          while (idx >= 0) {
-            sbpol.delete(idx, idx + (enclosure == null ? 0 : enclosure.length()));
-            idx = sbpol.indexOf(enclosure + enclosure);
-          }
-          pol = sbpol.toString();
-        }
-
-        //  replace the escaped enclosures with enclosures...
-        if (containsEscapedEnclosures) {
-          String replace = escapeCharacter + enclosure;
-          String replaceWith = enclosure;
-
-          pol = Const.replace(pol, replace, replaceWith);
-        }
-
-        //replace the escaped separators with separators...
-        if (containsEscapedSeparators) {
-          String replace = escapeCharacter + delimiter;
-          String replaceWith = delimiter;
-
-          pol = Const.replace(pol, replace, replaceWith);
-        }
-
-        // Now add pol to the strings found!
-        strings.add(pol);
-
-        pos = next + delimiter.length();
-      }
-      if (pos == length) {
-        if (log.isRowLevel()) {
-          log.logRowlevel(BaseMessages.getString(PKG, "CsvInput.Log.ConvertLineToRowTitle"),
-              BaseMessages.getString(PKG, "CsvInput.Log.EndOfEmptyLineFound"));
-        }
-        strings.add(""); //$NON-NLS-1$
-      }
-    } catch (Exception e) {
-      throw new KettleException(
-          BaseMessages.getString(PKG, "CsvInput.Log.Error.ErrorConvertingLine", e.toString()),
-          e); //$NON-NLS-1$
-    }
-
-    return strings.toArray(new String[strings.size()]);
-  }
-
   public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException {
     meta = (CsvInputMeta) smi;
     data = (CsvInputData) sdi;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/Queue.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/Queue.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/Queue.java
deleted file mode 100644
index 91c78ad..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/Queue.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.processing.dataprocessor.queue;
-
-public interface Queue<E> {
-  /**
-   * This method insert the element in the queue, and return false if it fails
-   * to isert in the queue.
-   *
-   * @param obj
-   * @return true if insererted properly false otherwise.
-   */
-  boolean offer(E obj);
-
-  /**
-   * This method get the element from the head of the queue and
-   * remove the element from the queue.
-   *
-   * @return
-   */
-  E poll();
-
-  /**
-   * This method just return the element in the head of the queue.
-   *
-   * @return
-   */
-  E peek();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/DataProcessorQueue.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/DataProcessorQueue.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/DataProcessorQueue.java
deleted file mode 100644
index d73700e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/DataProcessorQueue.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.dataprocessor.queue.impl;
-
-import java.util.concurrent.PriorityBlockingQueue;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.carbondata.processing.dataprocessor.queue.Queue;
-import org.apache.carbondata.processing.dataprocessor.record.holder.DataProcessorRecordHolder;
-
-public class DataProcessorQueue implements Queue<DataProcessorRecordHolder> {
-  /**
-   * Size of the queue
-   */
-  private int qSize;
-
-  /**
-   * Counter to maintain state of the queue.
-   */
-  private AtomicInteger counter;
-
-  /**
-   * Queue that holds the data.
-   */
-  private PriorityBlockingQueue<DataProcessorRecordHolder> priorityQueue;
-
-  public DataProcessorQueue(int size) {
-    this.counter = new AtomicInteger();
-    this.qSize = size;
-    this.priorityQueue =
-        new PriorityBlockingQueue<DataProcessorRecordHolder>(size, new RecordComparator());
-
-  }
-
-  @Override public boolean offer(DataProcessorRecordHolder obj) {
-    if (counter.get() == qSize) {
-      return false;
-    } else {
-
-      priorityQueue.offer(obj);
-      counter.getAndIncrement();
-      return true;
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override public DataProcessorRecordHolder poll() {
-    if (priorityQueue.isEmpty()) {
-      return null;
-    } else {
-      counter.getAndDecrement();
-      return (DataProcessorRecordHolder) priorityQueue.poll();
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override public DataProcessorRecordHolder peek() {
-    if (priorityQueue.isEmpty()) {
-      return null;
-    } else {
-      return priorityQueue.peek();
-    }
-  }
-
-  /**
-   * Is Queue is Full.
-   *
-   * @return
-   */
-  public boolean isFull() {
-    return counter.get() == qSize;
-  }
-
-  /**
-   * Is queue is Empty
-   *
-   * @return
-   */
-  public boolean isEmpty() {
-    return priorityQueue.isEmpty();
-  }
-
-  /**
-   * return the size (i.e. Elements present in the Queue)
-   */
-  public int size() {
-    return counter.get();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java
index d34803a..f7994dc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.java
@@ -24,34 +24,12 @@ package org.apache.carbondata.processing.dataprocessor.record.holder;
  */
 public class DataProcessorRecordHolder {
 
-  private Object[][] processedRecords;
-
-  private Object[][] originalRecords;
-
   private int seqNumber;
 
-  private int counter;
-
-  private int processCounter;
-
   public DataProcessorRecordHolder(int size, int seqNumber) {
-    this.originalRecords = new Object[size][];
-    this.processedRecords = new Object[size][];
     this.seqNumber = seqNumber;
   }
 
-  public Object[][] getOriginalRow() {
-    return originalRecords;
-  }
-
-  public void addRow(Object[] oriRow) {
-    originalRecords[counter++] = oriRow;
-  }
-
-  public void addProcessedRows(Object[] processedRows) {
-    processedRecords[processCounter++] = processedRows;
-  }
-
   /**
    * Returns the sequence number.
    */
@@ -59,7 +37,4 @@ public class DataProcessorRecordHolder {
     return seqNumber;
   }
 
-  public Object[][] getProcessedRow() {
-    return processedRecords;
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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 9843c2e..14553ec 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
@@ -110,14 +110,6 @@ public class ArrayDataType implements GenericDataType<ArrayObject> {
   }
 
   /*
-   * set column name
-   */
-  @Override
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  /*
    * set parent name
    */
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java b/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java
index f8c765b..1be004b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/datatypes/GenericDataType.java
@@ -44,11 +44,6 @@ public interface GenericDataType<T> {
   String getName();
 
   /**
-   * @param name - To set column name
-   */
-  void setName(String name);
-
-  /**
    * @return - columns parent name
    */
   String getParentname();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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 610366c..38f8c79 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
@@ -155,14 +155,6 @@ public class PrimitiveDataType implements GenericDataType<Object> {
   }
 
   /*
-   * set column name
-   */
-  @Override
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  /*
    * get column parent name
    */
   @Override
@@ -245,9 +237,6 @@ public class PrimitiveDataType implements GenericDataType<Object> {
     dimCardWithComplex.add(dictionaryGenerator.size());
   }
 
-  /*
-       * parse bytearray and bit pack
-       */
   @Override
   public void parseAndBitPack(ByteBuffer byteArrayInput, DataOutputStream dataOutputStream,
       KeyGenerator[] generator) throws IOException, KeyGenException {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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 f034895..09cd421 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
@@ -100,14 +100,6 @@ public class StructDataType implements GenericDataType<StructObject> {
   }
 
   /*
-   * set column name
-   */
-  @Override
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  /*
    * get parent column name
    */
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
index eb10549..fb9599e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
@@ -207,16 +207,13 @@ public class GraphGenerator {
 
   private String rddIteratorKey;
 
-  public GraphGenerator(DataLoadModel dataLoadModel, boolean isHDFSReadMode, String partitionID,
-      String factStoreLocation, int allocate,
+  public GraphGenerator(DataLoadModel dataLoadModel, String partitionID, String factStoreLocation,
       CarbonDataLoadSchema carbonDataLoadSchema, String segmentId) {
     CarbonMetadata.getInstance().addCarbonTable(carbonDataLoadSchema.getCarbonTable());
     this.schemaInfo = dataLoadModel.getSchemaInfo();
     this.tableName = dataLoadModel.getTableName();
     this.isCSVLoad = dataLoadModel.isCsvLoad();
-    this.modifiedDimension = dataLoadModel.getModifiedDimesion();
     this.isAutoAggRequest = schemaInfo.isAutoAggregateRequest();
-    //this.schema = schema;
     this.carbonDataLoadSchema = carbonDataLoadSchema;
     this.databaseName = carbonDataLoadSchema.getCarbonTable().getDatabaseName();
     this.partitionID = partitionID;
@@ -235,11 +232,9 @@ public class GraphGenerator {
     LOGGER.info("************* Is Columnar Storage" + isColumnar);
   }
 
-  public GraphGenerator(DataLoadModel dataLoadModel, boolean isHDFSReadMode, String partitionID,
-      String factStoreLocation, int allocate, CarbonDataLoadSchema carbonDataLoadSchema,
-      String segmentId, String outputLocation) {
-    this(dataLoadModel, isHDFSReadMode, partitionID, factStoreLocation, allocate,
-        carbonDataLoadSchema, segmentId);
+  public GraphGenerator(DataLoadModel dataLoadModel, String partitionID, String factStoreLocation,
+      CarbonDataLoadSchema carbonDataLoadSchema, String segmentId, String outputLocation) {
+    this(dataLoadModel, partitionID, factStoreLocation, carbonDataLoadSchema, segmentId);
     this.outputLocation = outputLocation;
     this.rddIteratorKey = dataLoadModel.getRddIteratorKey();
   }
@@ -561,7 +556,6 @@ public class GraphGenerator {
     seqMeta.setDatabaseName(schemaInfo.getDatabaseName());
     seqMeta.setComplexDelimiterLevel1(schemaInfo.getComplexDelimiterLevel1());
     seqMeta.setComplexDelimiterLevel2(schemaInfo.getComplexDelimiterLevel2());
-    seqMeta.setCarbonMetaHier(graphConfiguration.getMetaHeirString());
     seqMeta.setCarbonmsr(graphConfiguration.getMeasuresString());
     seqMeta.setCarbonProps(graphConfiguration.getPropertiesString());
     seqMeta.setCarbonhier(graphConfiguration.getHiersString());
@@ -823,7 +817,7 @@ public class GraphGenerator {
     graphConfiguration.setNoDictionaryDims(noDictionarydimString.toString());
 
     String tableString =
-        CarbonSchemaParser.getTableNameString(factTableName, dimensions, carbonDataLoadSchema);
+        CarbonSchemaParser.getTableNameString(dimensions, carbonDataLoadSchema);
     String dimensionColumnIds = CarbonSchemaParser.getColumnIdString(dimensions);
     graphConfiguration.setDimensionTableNames(tableString);
     graphConfiguration.setDimensionString(dimString.toString());
@@ -889,9 +883,6 @@ public class GraphGenerator {
     // check quotes required in query or Not
     boolean isQuotesRequired = true;
     String quote = CarbonSchemaParser.QUOTES;
-    if (null != schemaInfo.getSrcDriverName()) {
-      quote = getQuoteType(schemaInfo);
-    }
     graphConfiguration.setTableInputSqlQuery(CarbonSchemaParser
         .getTableInputSQLQuery(dimensions, measures,
             carbonDataLoadSchema.getCarbonTable().getFactTableName(), isQuotesRequired,
@@ -908,10 +899,6 @@ public class GraphGenerator {
 
     graphConfiguration.setMeasures(CarbonSchemaParser.getMeasures(measures));
     graphConfiguration.setAGG(false);
-    graphConfiguration.setUsername(schemaInfo.getSrcUserName());
-    graphConfiguration.setPassword(schemaInfo.getSrcPwd());
-    graphConfiguration.setDriverclass(schemaInfo.getSrcDriverName());
-    graphConfiguration.setConnectionUrl(schemaInfo.getSrcConUrl());
     return graphConfiguration;
   }
 
@@ -928,25 +915,6 @@ public class GraphGenerator {
     return tableOptionWrapper;
   }
 
-  private String getQuoteType(SchemaInfo schemaInfo) throws GraphGeneratorException {
-    String driverClass = schemaInfo.getSrcDriverName();
-    String type = DRIVERS.get(driverClass);
-
-    if (null == type) {
-      LOGGER.error("Driver : \"" + driverClass + " \"Not Supported.");
-      throw new GraphGeneratorException("Driver : \"" + driverClass + " \"Not Supported.");
-    }
-
-    if (type.equals(CarbonCommonConstants.TYPE_ORACLE) || type
-        .equals(CarbonCommonConstants.TYPE_MSSQL)) {
-      return CarbonSchemaParser.QUOTES;
-    } else if (type.equals(CarbonCommonConstants.TYPE_MYSQL)) {
-      return CarbonSchemaParser.QUOTES;
-    }
-
-    return CarbonSchemaParser.QUOTES;
-  }
-
   public CarbonTable getTable() {
     return carbonDataLoadSchema.getCarbonTable();
   }
@@ -959,7 +927,6 @@ public class GraphGenerator {
    */
   private void prepareNoDictionaryMapping(List<CarbonDimension> dims,
       GraphConfigurationInfo graphConfig) {
-    // boolean[] NoDictionaryMapping = new boolean[dims.size()];
     List<Boolean> noDictionaryMapping = new ArrayList<Boolean>();
     for (CarbonDimension dimension : dims) {
       // for  complex type need to break the loop

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
index 7c428a7..d1ada49 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
@@ -228,13 +228,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return the connectionName
-   */
-  public String getConnectionName() {
-    return connectionName;
-  }
-
-  /**
    * @param connectionName the connectionName to set
    */
   public void setConnectionName(String connectionName) {
@@ -242,20 +235,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return the dbType
-   */
-  public String getDbType() {
-    return dbType;
-  }
-
-  /**
-   * @param dbType the dbType to set
-   */
-  public void setDbType(String dbType) {
-    this.dbType = dbType;
-  }
-
-  /**
    * @return the numberOfCores
    */
   public String getNumberOfCores() {
@@ -270,13 +249,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return the storeLocation
-   */
-  public String getStoreLocation() {
-    return storeLocation;
-  }
-
-  /**
    * @param storeLocation the storeLocation to set
    */
   public void setStoreLocation(String storeLocation) {
@@ -306,13 +278,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return the blockletSize
-   */
-  public String getBlockletSize() {
-    return blockletSize;
-  }
-
-  /**
    * @param blockletSize the blockletSize to set
    */
   public void setBlockletSize(String blockletSize) {
@@ -320,13 +285,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return the maxBlockletInFile
-   */
-  public String getMaxBlockletInFile() {
-    return maxBlockletInFile;
-  }
-
-  /**
    * @param maxBlockletInFile the maxBlockletInFile to set
    */
   public void setMaxBlockletInFile(String maxBlockletInFile) {
@@ -413,15 +371,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * setNormHiers
-   *
-   * @param normHiers void
-   */
-  public void setNormHiers(String normHiers) {
-    this.normHiers = normHiers;
-  }
-
-  /**
    * @return the hiersString
    */
   public String getHiersString() {
@@ -457,27 +406,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @param propertiesString the propertiesString to set
-   */
-  public void setPropertiesString(String propertiesString) {
-    this.propertiesString = propertiesString;
-  }
-
-  /**
-   * @return the timeHeirString
-   */
-  public String getTimeHeirString() {
-    return timeHeirString;
-  }
-
-  /**
-   * @param timeHeirString the timeHeirString to set
-   */
-  public void setTimeHeirString(String timeHeirString) {
-    this.timeHeirString = timeHeirString;
-  }
-
-  /**
    * @return the metaHeirString
    */
   public String getMetaHeirString() {
@@ -492,48 +420,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return the metaHeirQueryString
-   */
-  public String getMetaHeirQueryString() {
-    return metaHeirQueryString;
-  }
-
-  /**
-   * @param metaHeirQueryString the metaHeirQueryString to set
-   */
-  public void setMetaHeirQueryString(String metaHeirQueryString) {
-    this.metaHeirQueryString = metaHeirQueryString;
-  }
-
-  /**
-   * @return the jndiName
-   */
-  public String getJndiName() {
-    return jndiName;
-  }
-
-  /**
-   * @param jndiName the jndiName to set
-   */
-  public void setJndiName(String jndiName) {
-    this.jndiName = jndiName;
-  }
-
-  /**
-   * @return the tableMeasuresAndDataTypeMap
-   */
-  public Map<String, String> getTableMeasuresAndDataTypeMap() {
-    return tableMeasuresAndDataTypeMap;
-  }
-
-  /**
-   * @param tableMeasuresAndDataTypeMap the tableMeasuresAndDataTypeMap to set
-   */
-  public void setTableMeasuresAndDataTypeMap(Map<String, String> tableMeasuresAndDataTypeMap) {
-    this.tableMeasuresAndDataTypeMap = tableMeasuresAndDataTypeMap;
-  }
-
-  /**
    * @return the tableInputSqlQuery
    */
   public String getTableInputSqlQuery() {
@@ -562,13 +448,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return the sortSize
-   */
-  public String getSortSize() {
-    return sortSize;
-  }
-
-  /**
    * @param sortSize the sortSize to set
    */
   public void setSortSize(String sortSize) {
@@ -756,13 +635,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return Returns the forignKey.
-   */
-  public String[] getForignKey() {
-    return forignKey;
-  }
-
-  /**
    * @param forignKey The forignKey to set.
    */
   public void setForignKey(String[] forignKey) {
@@ -889,10 +761,6 @@ public class GraphConfigurationInfo {
     return columnAndTableNameColumnMapForAgg;
   }
 
-  public void setColumnAndTableNameColumnMapForAgg(String columnAndTableNameColumnMapForAgg) {
-    this.columnAndTableNameColumnMapForAgg = columnAndTableNameColumnMapForAgg;
-  }
-
   /**
    * @return Returns the denormColumns.
    */
@@ -915,20 +783,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @param aggClass the aggClass to set
-   */
-  public void setAggClass(String[] aggClass) {
-    this.aggClass = aggClass;
-  }
-
-  /**
-   * @return the measureUniqueColumnNamesString
-   */
-  public String getMeasureUniqueColumnNamesString() {
-    return measureUniqueColumnNamesString;
-  }
-
-  /**
    * @param measureUniqueColumnNamesString the measureUniqueColumnNamesString to set
    */
   public void setMeasureUniqueColumnNamesString(String measureUniqueColumnNamesString) {
@@ -936,13 +790,6 @@ public class GraphConfigurationInfo {
   }
 
   /**
-   * @return the type
-   */
-  public char[] getType() {
-    return type;
-  }
-
-  /**
    * @param type the type to set
    */
   public void setType(char[] type) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java
index a2f0c59..559f363 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java
@@ -295,10 +295,6 @@ public class MDKeyGenStepMeta extends BaseStepMeta implements StepMetaInterface
     this.tableName = tableName;
   }
 
-  public String getAggregateLevels() {
-    return aggregateLevels;
-  }
-
   public void setAggregateLevels(String aggregateLevels) {
     this.aggregateLevels = aggregateLevels;
   }
@@ -307,14 +303,6 @@ public class MDKeyGenStepMeta extends BaseStepMeta implements StepMetaInterface
     return complexTypes;
   }
 
-  public void setComplexTypes(Map<String, GenericDataType> complexTypes) {
-    this.complexTypes = complexTypes;
-  }
-
-  public String getNumberOfCores() {
-    return numberOfCores;
-  }
-
   public void setNumberOfCores(String numberOfCores) {
     this.numberOfCores = numberOfCores;
   }
@@ -376,13 +364,6 @@ public class MDKeyGenStepMeta extends BaseStepMeta implements StepMetaInterface
   }
 
   /**
-   * @return the complexTypeString
-   */
-  public int getComplexTypeString() {
-    return Integer.parseInt(complexTypeString);
-  }
-
-  /**
    * @param complexTypeString the complexTypeString to set
    */
   public void setComplexTypeString(String complexTypeString) {
@@ -417,10 +398,6 @@ public class MDKeyGenStepMeta extends BaseStepMeta implements StepMetaInterface
     this.noDictionaryCount = noDictionaryCount;
   }
 
-  public String getColumnGroupsString() {
-    return this.columnGroupsString;
-  }
-
   public void setColumnGroupsString(String columnGroups) {
     this.columnGroupsString = columnGroups;
 
@@ -461,10 +438,6 @@ public class MDKeyGenStepMeta extends BaseStepMeta implements StepMetaInterface
     return complexTypesMap;
   }
 
-  public String getMeasureDataType() {
-    return measureDataType;
-  }
-
   public void setMeasureDataType(String measureDataType) {
     this.measureDataType = measureDataType;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileManager.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileManager.java b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileManager.java
index 7182701..8cece09 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileManager.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/file/FileManager.java
@@ -46,13 +46,6 @@ public class FileManager implements IFileManagerComposite {
     return listOfFileData.get(i);
   }
 
-  /**
-   * Renames the File/Folders
-   */
-  public boolean rename(IFileManagerComposite composite) {
-    return false;
-  }
-
   @Override public void setName(String name) {
     this.fileName = name;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java b/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java
index d98d40a..68e7d2a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java
@@ -22,8 +22,6 @@ package org.apache.carbondata.processing.merger.step;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
 import org.pentaho.di.core.CheckResult;
 import org.pentaho.di.core.CheckResultInterface;
 import org.pentaho.di.core.Counter;
@@ -357,15 +355,6 @@ public class CarbonSliceMergerStepMeta extends BaseStepMeta
   }
 
   /**
-   * This method will return mdkey size
-   *
-   * @return mdkey size
-   */
-  public String getMdkeySize() {
-    return mdkeySize;
-  }
-
-  /**
    * This method will be used to set the mdkey
    *
    * @param mdkeySize
@@ -375,15 +364,6 @@ public class CarbonSliceMergerStepMeta extends BaseStepMeta
   }
 
   /**
-   * This method will be used to get the measure count
-   *
-   * @return measure count
-   */
-  public String getMeasureCount() {
-    return measureCount;
-  }
-
-  /**
    * This method will be used to set the measure count
    *
    * @param measureCount
@@ -393,15 +373,6 @@ public class CarbonSliceMergerStepMeta extends BaseStepMeta
   }
 
   /**
-   * This method will be used to get the heir and its key suze string
-   *
-   * @return heirAndKeySize
-   */
-  public String getHeirAndKeySize() {
-    return heirAndKeySize;
-  }
-
-  /**
    * This method will be used to set the heir and key size string
    *
    * @param heirAndKeySize
@@ -439,13 +410,6 @@ public class CarbonSliceMergerStepMeta extends BaseStepMeta
   }
 
   /**
-   * @return the isGroupByEnabled
-   */
-  public boolean isGroupByEnabled() {
-    return Boolean.parseBoolean(groupByEnabled);
-  }
-
-  /**
    * @param isGroupByEnabled the isGroupByEnabled to set
    */
   public void setGroupByEnabled(String isGroupByEnabled) {
@@ -453,27 +417,6 @@ public class CarbonSliceMergerStepMeta extends BaseStepMeta
   }
 
   /**
-   * @return the aggregators
-   */
-  public String[] getAggregators() {
-    return aggregatorString.split(CarbonCommonConstants.HASH_SPC_CHARACTER);
-  }
-
-  /**
-   * @return the aggregatorClass
-   */
-  public String[] getAggregatorClass() {
-    return aggregatorClassString.split(CarbonCommonConstants.HASH_SPC_CHARACTER);
-  }
-
-  /**
-   * @return the aggregatorString
-   */
-  public String getAggregatorString() {
-    return aggregatorString;
-  }
-
-  /**
    * @param aggregatorString the aggregatorString to set
    */
   public void setAggregatorString(String aggregatorString) {
@@ -481,13 +424,6 @@ public class CarbonSliceMergerStepMeta extends BaseStepMeta
   }
 
   /**
-   * @return the aggregatorClassString
-   */
-  public String getAggregatorClassString() {
-    return aggregatorClassString;
-  }
-
-  /**
    * @param aggregatorClassString the aggregatorClassString to set
    */
   public void setAggregatorClassString(String aggregatorClassString) {
@@ -495,35 +431,17 @@ public class CarbonSliceMergerStepMeta extends BaseStepMeta
   }
 
   /**
-   * @return the factDimLensString
-   */
-  public String getFactDimLensString() {
-    return factDimLensString;
-  }
-
-  /**
    * @param factDimLensString1 the factDimLensString to set
    */
   public void setFactDimLensString(String factDimLensString1) {
     this.factDimLensString = factDimLensString1;
   }
 
-  public String getLevelAnddataTypeString() {
-    return levelAnddataTypeString;
-  }
-
   public void setLevelAnddataTypeString(String levelAnddataTypeString) {
     this.levelAnddataTypeString = levelAnddataTypeString;
   }
 
   /**
-   * @return partitionId
-   */
-  public String getPartitionID() {
-    return partitionID;
-  }
-
-  /**
    * @param partitionID
    */
   public void setPartitionID(String partitionID) {
@@ -531,14 +449,6 @@ public class CarbonSliceMergerStepMeta extends BaseStepMeta
   }
 
   /**
-   * return segmentId
-   * @return
-   */
-  public int getSegmentId() {
-    return Integer.parseInt(segmentId);
-  }
-
-  /**
    * set segment Id
    * @param segmentId
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 692e5d7..9e8f5b0 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/model/CarbonLoadModel.java
@@ -211,10 +211,6 @@ public class CarbonLoadModel implements Serializable {
     return factFilesToProcess;
   }
 
-  public void setFactFilesToProcess(List<String> factFilesToProcess) {
-    this.factFilesToProcess = factFilesToProcess;
-  }
-
   public String getCsvHeader() {
     return csvHeader;
   }
@@ -402,13 +398,6 @@ public class CarbonLoadModel implements Serializable {
   }
 
   /**
-   * @return the aggTables
-   */
-  public String[] getAggTables() {
-    return aggTables;
-  }
-
-  /**
    * @param aggTables the aggTables to set
    */
   public void setAggTables(String[] aggTables) {
@@ -467,13 +456,6 @@ public class CarbonLoadModel implements Serializable {
   }
 
   /**
-   * @param isRetentionRequest
-   */
-  public void setRetentionRequest(boolean isRetentionRequest) {
-    this.isRetentionRequest = isRetentionRequest;
-  }
-
-  /**
    * getLoadMetadataDetails.
    *
    * @return

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 26300d6..79e17e2 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/CarbonDataLoadConfiguration.java
@@ -133,14 +133,6 @@ public class CarbonDataLoadConfiguration {
     dataLoadProperties.put(key, value);
   }
 
-  public Object getDataLoadProperty(String key, Object defaultValue) {
-    Object value = dataLoadProperties.get(key);
-    if (value == null) {
-      value = defaultValue;
-    }
-    return value;
-  }
-
   public Object getDataLoadProperty(String key) {
     return dataLoadProperties.get(key);
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index c5dc5d1..07b4fa1 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/DataField.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/newflow/DataField.java
@@ -21,7 +21,6 @@ package org.apache.carbondata.processing.newflow;
 
 import java.io.Serializable;
 
-import org.apache.carbondata.core.carbon.metadata.blocklet.compressor.CompressionCodec;
 import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
 
@@ -36,8 +35,6 @@ public class DataField implements Serializable {
 
   private CarbonColumn column;
 
-  private CompressionCodec compressionCodec;
-
   private String dateFormat;
 
   public boolean hasDictionaryEncoding() {
@@ -48,14 +45,6 @@ public class DataField implements Serializable {
     return column;
   }
 
-  public CompressionCodec getCompressionCodec() {
-    return compressionCodec;
-  }
-
-  public void setCompressionCodec(CompressionCodec compressionCodec) {
-    this.compressionCodec = compressionCodec;
-  }
-
   public String getDateFormat() {
     return dateFormat;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/newflow/iterator/InputIterator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/newflow/iterator/InputIterator.java b/processing/src/main/java/org/apache/carbondata/processing/newflow/iterator/InputIterator.java
deleted file mode 100644
index a87aaa2..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/newflow/iterator/InputIterator.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.processing.newflow.iterator;
-
-import java.util.Iterator;
-
-/**
- * It iterates the data of record readers
- */
-public abstract class InputIterator<E> implements Iterator<E> {
-
-  @Override
-  public void remove() {
-    throw new UnsupportedOperationException("Not permitted");
-  }
-
-  /**
-   * Initialize the iterator
-   */
-  public abstract void initialize();
-
-  /**
-   * Close the resources
-   */
-  public abstract void close();
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 5d12ec4..036d15a 100644
--- 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
@@ -64,10 +64,7 @@ public class ParallelReadMergeSorterImpl implements Sorter {
 
   private SingleThreadFinalSortFilesMerger finalMerger;
 
-  private DataField[] inputDataFields;
-
   public ParallelReadMergeSorterImpl(DataField[] inputDataFields) {
-    this.inputDataFields = inputDataFields;
   }
 
   @Override
@@ -187,15 +184,12 @@ public class ParallelReadMergeSorterImpl implements Sorter {
 
     private SortDataRows sortDataRows;
 
-    private SortParameters parameters;
-
     private Object[][] buffer;
 
     public SortIteratorThread(Iterator<CarbonRowBatch> iterator, SortDataRows sortDataRows,
         SortParameters parameters, int batchSize) {
       this.iterator = iterator;
       this.sortDataRows = sortDataRows;
-      this.parameters = parameters;
       this.buffer = new Object[batchSize][];
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 3a29647..4504f0c 100644
--- 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
@@ -64,10 +64,7 @@ public class UnsafeParallelReadMergeSorterImpl implements Sorter {
 
   private UnsafeSingleThreadFinalSortFilesMerger finalMerger;
 
-  private DataField[] inputDataFields;
-
   public UnsafeParallelReadMergeSorterImpl(DataField[] inputDataFields) {
-    this.inputDataFields = inputDataFields;
   }
 
   @Override public void initialize(SortParameters sortParameters) {
@@ -180,15 +177,12 @@ public class UnsafeParallelReadMergeSorterImpl implements Sorter {
 
     private UnsafeSortDataRows sortDataRows;
 
-    private SortParameters parameters;
-
     private Object[][] buffer;
 
     public SortIteratorThread(Iterator<CarbonRowBatch> iterator, UnsafeSortDataRows sortDataRows,
         SortParameters parameters, int batchSize) {
       this.iterator = iterator;
       this.sortDataRows = sortDataRows;
-      this.parameters = parameters;
       this.buffer = new Object[batchSize][];
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 7296e74..0adce19 100644
--- 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
@@ -20,9 +20,7 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.util.Arrays;
-import java.util.Iterator;
 
-import org.apache.carbondata.common.CarbonIterator;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.memory.MemoryBlock;
 import org.apache.carbondata.core.unsafe.CarbonUnsafe;
@@ -73,10 +71,6 @@ public class UnsafeCarbonRowPage {
     lastSize = lastSize + size;
   }
 
-  public Iterator<Object[]> getIterator() {
-    return new UnsafeIterator();
-  }
-
   private int addRow(Object[] row, long address) {
     if (row == null) {
       throw new RuntimeException("Row is null ??");
@@ -310,30 +304,6 @@ public class UnsafeCarbonRowPage {
     return dataBlock;
   }
 
-  class UnsafeIterator extends CarbonIterator<Object[]> {
-
-    private int counter;
-
-    private int actualSize;
-
-    public UnsafeIterator() {
-      this.actualSize = buffer.getActualSize();
-    }
-
-    @Override public boolean hasNext() {
-      if (counter < actualSize) {
-        return true;
-      }
-      return false;
-    }
-
-    @Override public Object[] next() {
-      long address = buffer.get(counter);
-      counter++;
-      return getRow(address + dataBlock.getBaseOffset());
-    }
-  }
-
   public static void set(long[] words, int index) {
     int wordOffset = (index >> 6);
     words[wordOffset] |= (1L << index);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 9d73ba2..214e574 100644
--- 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
@@ -43,9 +43,6 @@ 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.sortandgroupby.sortdata.SortTempFileChunkWriter;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.TempSortFileWriter;
-import org.apache.carbondata.processing.sortandgroupby.sortdata.TempSortFileWriterFactory;
 import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
 
 public class UnsafeSortDataRows {
@@ -201,24 +198,6 @@ public class UnsafeSortDataRows {
     startFileBasedMerge();
   }
 
-  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(UnsafeCarbonRowPage rowPage, File file)
       throws CarbonSortKeyAndGroupByException {
     DataOutputStream stream = null;
@@ -242,24 +221,6 @@ public class UnsafeSortDataRows {
     }
   }
 
-  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
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 9f157a0..2c91f8f 100644
--- 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
@@ -76,10 +76,6 @@ public class UnsafeInmemoryMergeHolder implements Comparable<UnsafeInmemoryMerge
     return comparator.compare(currentRow, baseObject, o.getRow(), o.getBaseObject());
   }
 
-  public int numberOfRows() {
-    return actualSize;
-  }
-
   public Object getBaseObject() {
     return baseObject;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index 0d36d90..199a83e 100644
--- 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
@@ -73,7 +73,6 @@ public class UnsafeInMemoryIntermediateDataMerger implements Callable<Void> {
   @Override public Void call() throws Exception {
     long intermediateMergeStartTime = System.currentTimeMillis();
     int holderCounterConst = holderCounter;
-    boolean isFailed = false;
     try {
       startSorting();
       while (hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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 a142823..0eaccaf 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
@@ -45,11 +45,6 @@ public class UnsafeSingleThreadFinalSortFilesMerger extends CarbonIterator<Objec
       LogServiceFactory.getLogService(UnsafeSingleThreadFinalSortFilesMerger.class.getName());
 
   /**
-   * lockObject
-   */
-  private static final Object LOCKOBJECT = new Object();
-
-  /**
    * fileCounter
    */
   private int fileCounter;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/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
index b979af6..23f100e 100644
--- 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
@@ -10,8 +10,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 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.util.CarbonProperties;
 import org.apache.carbondata.processing.newflow.AbstractDataLoadProcessorStep;
 import org.apache.carbondata.processing.newflow.CarbonDataLoadConfiguration;
@@ -27,9 +25,6 @@ import org.apache.carbondata.processing.newflow.row.CarbonRowBatch;
  */
 public class InputProcessorStepImpl extends AbstractDataLoadProcessorStep {
 
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(InputProcessorStepImpl.class.getName());
-
   private RowParser rowParser;
 
   private CarbonIterator<Object[]>[] inputIterators;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java
index 43a4b8a..de94724 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java
@@ -216,38 +216,17 @@ public class ColumnsInfo {
   }
 
   /**
-   * @return Returns the batchSize.
-   */
-  public int getBatchSize() {
-    return batchSize;
-  }
-
-  /**
    * @param batchSize The batchSize to set.
    */
   public void setBatchSize(int batchSize) {
     this.batchSize = batchSize;
   }
 
-  /**
-   * @return Returns the isInitialLoad.
-   */
-  public boolean isAggregateLoad() {
-    return isAggregateLoad;
-  }
-
   public void setAggregateLoad(boolean isAggregateLoad) {
     this.isAggregateLoad = isAggregateLoad;
   }
 
   /**
-   * @return Returns the storeType.
-   */
-  public String getStoreType() {
-    return storeType;
-  }
-
-  /**
    * @param storeType The storeType to set.
    */
   public void setStoreType(String storeType) {
@@ -297,13 +276,6 @@ public class ColumnsInfo {
   }
 
   /**
-   * @return Returns the propColumns.
-   */
-  public List<String>[] getPropColumns() {
-    return propColumns;
-  }
-
-  /**
    * @param propColumns The propColumns to set.
    */
   public void setPropColumns(List<String>[] propColumns) {
@@ -332,13 +304,6 @@ public class ColumnsInfo {
   }
 
   /**
-   * @param timDimIndexEnd The timDimIndexEnd to set.
-   */
-  public void setTimDimIndexEnd(int timDimIndexEnd) {
-    this.timDimIndexEnd = timDimIndexEnd;
-  }
-
-  /**
    * @return Returns the timeOrdinalIndices.
    */
   public int[] getTimeOrdinalIndices() {
@@ -353,13 +318,6 @@ public class ColumnsInfo {
   }
 
   /**
-   * @return Returns the timeOrdinalCols.
-   */
-  public String[] getTimeOrdinalCols() {
-    return timeOrdinalCols;
-  }
-
-  /**
    * @param timeOrdinalCols The timeOrdinalCols to set.
    */
   public void setTimeOrdinalCols(String[] timeOrdinalCols) {
@@ -367,13 +325,6 @@ public class ColumnsInfo {
   }
 
   /**
-   * @return Returns the propTypes.
-   */
-  public List<String>[] getPropTypes() {
-    return propTypes;
-  }
-
-  /**
    * @param propTypes The propTypes to set.
    */
   public void setPropTypes(List<String>[] propTypes) {
@@ -395,13 +346,6 @@ public class ColumnsInfo {
   }
 
   /**
-   * @return Returns the dimHierRel.
-   */
-  public String[] getDimHierRel() {
-    return dimHierRel;
-  }
-
-  /**
    * @param dimHierRel The dimHierRel to set.
    */
   public void setDimHierRel(String[] dimHierRel) {
@@ -432,22 +376,6 @@ public class ColumnsInfo {
   }
 
   /**
-   * setDimsPresent
-   *
-   * @param dimsPresent
-   */
-  public void setDimsPresent(boolean[] dimsPresent) {
-    this.dimsPresent = dimsPresent;
-  }
-
-  /**
-   * @return Returns the measureColumns.
-   */
-  public String[] getMeasureColumns() {
-    return measureColumns;
-  }
-
-  /**
    * @param measureColumns The measureColumns to set.
    */
   public void setMeasureColumns(String[] measureColumns) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3fe69037/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java
index d875cf2..2d51dab 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java
@@ -29,48 +29,10 @@ public class HierarchiesInfo {
   private String hierarichieName;
 
   /**
-   * columnIndex
-   */
-  private int[] columnIndex;
-
-  /**
-   * columnNames
-   */
-  private String[] columnNames;
-
-  /**
    * columnPropMap
    */
   private Map<String, String[]> columnPropMap;
 
-  /**
-   * loadToHierarichiTable
-   */
-  private boolean loadToHierarichiTable;
-
-  /**
-   * query
-   */
-  private String query;
-
-  /**
-   * Is Time Dimension
-   */
-  private boolean isTimeDimension;
-
-  /**
-   * levelTypeColumnMap
-   */
-  private Map<String, String> levelTypeColumnMap;
-
-  public boolean isLoadToHierarichiTable() {
-    return loadToHierarichiTable;
-  }
-
-  public void setLoadToHierarichiTable(boolean loadToHierarichiTable) {
-    this.loadToHierarichiTable = loadToHierarichiTable;
-  }
-
   public String getHierarichieName() {
     return hierarichieName;
   }
@@ -79,22 +41,6 @@ public class HierarchiesInfo {
     this.hierarichieName = hierarichieName;
   }
 
-  public int[] getColumnIndex() {
-    return columnIndex;
-  }
-
-  public void setColumnIndex(int[] columnIndex) {
-    this.columnIndex = columnIndex;
-  }
-
-  public String[] getColumnNames() {
-    return columnNames;
-  }
-
-  public void setColumnNames(String[] columnNames) {
-    this.columnNames = columnNames;
-  }
-
   public Map<String, String[]> getColumnPropMap() {
     return columnPropMap;
   }
@@ -103,28 +49,5 @@ public class HierarchiesInfo {
     this.columnPropMap = columnPropMap;
   }
 
-  public String getQuery() {
-    return query;
-  }
-
-  public void setQuery(String query) {
-    this.query = query;
-  }
-
-  public boolean isTimeDimension() {
-    return isTimeDimension;
-  }
-
-  public void setTimeDimension(boolean isTimeDimension) {
-    this.isTimeDimension = isTimeDimension;
-  }
-
-  public Map<String, String> getLevelTypeColumnMap() {
-    return levelTypeColumnMap;
-  }
-
-  public void setLevelTypeColumnMap(Map<String, String> levelTypeColumnMap) {
-    this.levelTypeColumnMap = levelTypeColumnMap;
-  }
 
 }