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/03/30 05:12:09 UTC

[01/13] incubator-carbondata git commit: Removed kettle related code and refactored

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 3d5cf456d -> 731908954


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
new file mode 100644
index 0000000..85f8470
--- /dev/null
+++ b/processing/src/test/java/org/apache/carbondata/processing/StoreCreator.java
@@ -0,0 +1,506 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.carbondata.processing;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.carbondata.common.CarbonIterator;
+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.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
+import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
+import org.apache.carbondata.core.metadata.ColumnIdentifier;
+import org.apache.carbondata.core.metadata.converter.SchemaConverter;
+import org.apache.carbondata.core.metadata.converter.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;
+import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.TableSchema;
+import org.apache.carbondata.core.metadata.schema.table.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.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
+import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
+import org.apache.carbondata.core.writer.ThriftWriter;
+import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
+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 com.google.gson.Gson;
+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;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
+
+/**
+ * This class will create store file based on provided schema
+ *
+ */
+public class StoreCreator {
+
+  private static AbsoluteTableIdentifier absoluteTableIdentifier;
+
+  static {
+    try {
+      String storePath = new File("target/store").getCanonicalPath();
+      String dbName = "testdb";
+      String tableName = "testtable";
+      absoluteTableIdentifier =
+          new AbsoluteTableIdentifier(storePath, new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
+    } catch (IOException ex) {
+
+    }
+  }
+
+  public static AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
+    return absoluteTableIdentifier;
+  }
+
+  /**
+   * Create store without any restructure
+   */
+  public static void createCarbonStore() {
+
+    try {
+
+      String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
+      File storeDir = new File(absoluteTableIdentifier.getStorePath());
+      CarbonUtil.deleteFoldersAndFiles(storeDir);
+      CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS,
+          absoluteTableIdentifier.getStorePath());
+
+      CarbonTable table = createTable();
+      writeDictionary(factFilePath, table);
+      CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
+      CarbonLoadModel loadModel = new CarbonLoadModel();
+      String partitionId = "0";
+      loadModel.setCarbonDataLoadSchema(schema);
+      loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+      loadModel.setFactFilePath(factFilePath);
+      loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
+      loadModel.setStorePath(absoluteTableIdentifier.getStorePath());
+      loadModel.setDateFormat(null);
+      loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
+          CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+      loadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty(
+          CarbonCommonConstants.CARBON_DATE_FORMAT,
+          CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
+      loadModel
+          .setSerializationNullFormat(
+              TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N");
+      loadModel
+          .setBadRecordsLoggerEnable(
+              TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false");
+      loadModel
+          .setBadRecordsAction(
+              TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "FORCE");
+      loadModel
+          .setIsEmptyDataBadRecord(
+              DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + "false");
+      loadModel.setCsvHeader("ID,date,country,name,phonetype,serialname,salary");
+      loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(","));
+      loadModel.setTaskNo("0");
+      loadModel.setSegmentId("0");
+      loadModel.setPartitionId("0");
+      loadModel.setFactTimeStamp(System.currentTimeMillis());
+
+      executeGraph(loadModel, absoluteTableIdentifier.getStorePath());
+
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+
+  }
+
+  private static CarbonTable createTable() throws IOException {
+    TableInfo tableInfo = new TableInfo();
+    tableInfo.setStorePath(absoluteTableIdentifier.getStorePath());
+    tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+    TableSchema tableSchema = new TableSchema();
+    tableSchema.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
+    ArrayList<Encoding> encodings = new ArrayList<>();
+    encodings.add(Encoding.DICTIONARY);
+    ColumnSchema id = new ColumnSchema();
+    id.setColumnName("ID");
+    id.setColumnar(true);
+    id.setDataType(DataType.INT);
+    id.setEncodingList(encodings);
+    id.setColumnUniqueId(UUID.randomUUID().toString());
+    id.setDimensionColumn(true);
+    id.setColumnGroup(1);
+    columnSchemas.add(id);
+
+    ColumnSchema date = new ColumnSchema();
+    date.setColumnName("date");
+    date.setColumnar(true);
+    date.setDataType(DataType.STRING);
+    date.setEncodingList(encodings);
+    date.setColumnUniqueId(UUID.randomUUID().toString());
+    date.setDimensionColumn(true);
+    date.setColumnGroup(2);
+    columnSchemas.add(date);
+
+    ColumnSchema country = new ColumnSchema();
+    country.setColumnName("country");
+    country.setColumnar(true);
+    country.setDataType(DataType.STRING);
+    country.setEncodingList(encodings);
+    country.setColumnUniqueId(UUID.randomUUID().toString());
+    country.setDimensionColumn(true);
+    country.setColumnGroup(3);
+    columnSchemas.add(country);
+
+    ColumnSchema name = new ColumnSchema();
+    name.setColumnName("name");
+    name.setColumnar(true);
+    name.setDataType(DataType.STRING);
+    name.setEncodingList(encodings);
+    name.setColumnUniqueId(UUID.randomUUID().toString());
+    name.setDimensionColumn(true);
+    name.setColumnGroup(4);
+    columnSchemas.add(name);
+
+    ColumnSchema phonetype = new ColumnSchema();
+    phonetype.setColumnName("phonetype");
+    phonetype.setColumnar(true);
+    phonetype.setDataType(DataType.STRING);
+    phonetype.setEncodingList(encodings);
+    phonetype.setColumnUniqueId(UUID.randomUUID().toString());
+    phonetype.setDimensionColumn(true);
+    phonetype.setColumnGroup(5);
+    columnSchemas.add(phonetype);
+
+    ColumnSchema serialname = new ColumnSchema();
+    serialname.setColumnName("serialname");
+    serialname.setColumnar(true);
+    serialname.setDataType(DataType.STRING);
+    serialname.setEncodingList(encodings);
+    serialname.setColumnUniqueId(UUID.randomUUID().toString());
+    serialname.setDimensionColumn(true);
+    serialname.setColumnGroup(6);
+    columnSchemas.add(serialname);
+
+    ColumnSchema salary = new ColumnSchema();
+    salary.setColumnName("salary");
+    salary.setColumnar(true);
+    salary.setDataType(DataType.INT);
+    salary.setEncodingList(new ArrayList<Encoding>());
+    salary.setColumnUniqueId(UUID.randomUUID().toString());
+    salary.setDimensionColumn(false);
+    salary.setColumnGroup(7);
+    columnSchemas.add(salary);
+
+    tableSchema.setListOfColumns(columnSchemas);
+    SchemaEvolution schemaEvol = new SchemaEvolution();
+    schemaEvol.setSchemaEvolutionEntryList(new ArrayList<SchemaEvolutionEntry>());
+    tableSchema.setSchemaEvalution(schemaEvol);
+    tableSchema.setTableId(UUID.randomUUID().toString());
+    tableInfo.setTableUniqueName(
+        absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName() + "_"
+            + absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+    tableInfo.setLastUpdatedTime(System.currentTimeMillis());
+    tableInfo.setFactTable(tableSchema);
+    tableInfo.setAggregateTableList(new ArrayList<TableSchema>());
+
+    CarbonTablePath carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            absoluteTableIdentifier.getCarbonTableIdentifier());
+    String schemaFilePath = carbonTablePath.getSchemaFilePath();
+    String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
+    tableInfo.setMetaDataFilepath(schemaMetadataPath);
+    CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
+
+    SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
+    org.apache.carbondata.format.TableInfo thriftTableInfo = schemaConverter
+        .fromWrapperToExternalTableInfo(tableInfo, tableInfo.getDatabaseName(),
+            tableInfo.getFactTable().getTableName());
+    org.apache.carbondata.format.SchemaEvolutionEntry schemaEvolutionEntry =
+        new org.apache.carbondata.format.SchemaEvolutionEntry(tableInfo.getLastUpdatedTime());
+    thriftTableInfo.getFact_table().getSchema_evolution().getSchema_evolution_history()
+        .add(schemaEvolutionEntry);
+
+    FileFactory.FileType fileType = FileFactory.getFileType(schemaMetadataPath);
+    if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
+      FileFactory.mkdirs(schemaMetadataPath, fileType);
+    }
+
+    ThriftWriter thriftWriter = new ThriftWriter(schemaFilePath, false);
+    thriftWriter.open();
+    thriftWriter.write(thriftTableInfo);
+    thriftWriter.close();
+    return CarbonMetadata.getInstance().getCarbonTable(tableInfo.getTableUniqueName());
+  }
+
+  private static void writeDictionary(String factFilePath, CarbonTable table) throws Exception {
+    BufferedReader reader = new BufferedReader(new FileReader(factFilePath));
+    String header = reader.readLine();
+    String[] split = header.split(",");
+    List<CarbonColumn> allCols = new ArrayList<CarbonColumn>();
+    List<CarbonDimension> dims = table.getDimensionByTableName(table.getFactTableName());
+    allCols.addAll(dims);
+    List<CarbonMeasure> msrs = table.getMeasureByTableName(table.getFactTableName());
+    allCols.addAll(msrs);
+    Set<String>[] set = new HashSet[dims.size()];
+    for (int i = 0; i < set.length; i++) {
+      set[i] = new HashSet<String>();
+    }
+    String line = reader.readLine();
+    while (line != null) {
+      String[] data = line.split(",");
+      for (int i = 0; i < set.length; i++) {
+        set[i].add(data[i]);
+      }
+      line = reader.readLine();
+    }
+
+    Cache dictCache = CacheProvider.getInstance()
+        .createCache(CacheType.REVERSE_DICTIONARY, absoluteTableIdentifier.getStorePath());
+    for (int i = 0; i < set.length; i++) {
+      ColumnIdentifier columnIdentifier = new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
+      CarbonDictionaryWriter writer =
+          new CarbonDictionaryWriterImpl(absoluteTableIdentifier.getStorePath(),
+              absoluteTableIdentifier.getCarbonTableIdentifier(), columnIdentifier);
+      for (String value : set[i]) {
+        writer.write(value);
+      }
+      writer.close();
+      writer.commit();
+      Dictionary dict = (Dictionary) dictCache.get(
+          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier.getCarbonTableIdentifier(),
+        		  columnIdentifier, dims.get(i).getDataType()));
+      CarbonDictionarySortInfoPreparator preparator =
+          new CarbonDictionarySortInfoPreparator();
+      List<String> newDistinctValues = new ArrayList<String>();
+      CarbonDictionarySortInfo dictionarySortInfo =
+          preparator.getDictionarySortInfo(newDistinctValues, dict, dims.get(i).getDataType());
+      CarbonDictionarySortIndexWriter carbonDictionaryWriter =
+          new CarbonDictionarySortIndexWriterImpl(
+              absoluteTableIdentifier.getCarbonTableIdentifier(), columnIdentifier,
+              absoluteTableIdentifier.getStorePath());
+      try {
+        carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
+        carbonDictionaryWriter.writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());
+      } finally {
+        carbonDictionaryWriter.close();
+      }
+    }
+    reader.close();
+  }
+
+  /**
+   * Execute graph which will further load data
+   *
+   * @param loadModel
+   * @param storeLocation
+   * @throws Exception
+   */
+  public static void executeGraph(CarbonLoadModel loadModel, String storeLocation)
+      throws Exception {
+    new File(storeLocation).mkdirs();
+    String outPutLoc = storeLocation + "/etl";
+    String databaseName = loadModel.getDatabaseName();
+    String tableName = loadModel.getTableName();
+    String tempLocationKey = databaseName + '_' + tableName + "_1";
+    CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation);
+    CarbonProperties.getInstance().addProperty("store_output_location", outPutLoc);
+    CarbonProperties.getInstance().addProperty("send.signal.load", "false");
+    CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true");
+    CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1");
+    CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true");
+    CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true");
+    CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true");
+    CarbonProperties.getInstance().addProperty("high.cardinality.value", "100000");
+    CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false");
+    CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000");
+
+    String fileNamePrefix = "";
+
+    String graphPath =
+        outPutLoc + File.separator + loadModel.getDatabaseName() + File.separator + tableName
+            + File.separator + 0 + File.separator + 1 + File.separator + tableName + ".ktr";
+    File path = new File(graphPath);
+    if (path.exists()) {
+      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();
+    CSVInputFormat.setCommentCharacter(configuration, loadModel.getCommentChar());
+    CSVInputFormat.setCSVDelimiter(configuration, loadModel.getCsvDelimiter());
+    CSVInputFormat.setEscapeCharacter(configuration, loadModel.getEscapeChar());
+    CSVInputFormat.setHeaderExtractionEnabled(configuration, true);
+    CSVInputFormat.setQuoteCharacter(configuration, loadModel.getQuoteChar());
+    CSVInputFormat.setReadBufferSize(configuration, CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
+            CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
+
+    TaskAttemptContextImpl hadoopAttemptContext = new TaskAttemptContextImpl(configuration, new TaskAttemptID("", 1, TaskType.MAP, 0, 0));
+    CSVInputFormat format = new CSVInputFormat();
+
+    RecordReader<NullWritable, StringArrayWritable> recordReader =
+        format.createRecordReader(blockDetails, hadoopAttemptContext);
+
+    CSVRecordReaderIterator readerIterator = new CSVRecordReaderIterator(recordReader, blockDetails, hadoopAttemptContext);
+    new DataLoadExecutor().execute(loadModel,
+        storeLocation,
+        new CarbonIterator[]{readerIterator});
+
+    info.setDatabaseName(databaseName);
+    info.setTableName(tableName);
+
+//    DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus);
+//    graphExecuter
+//        .executeGraph(graphPath, info, loadModel.getSchema());
+    //    LoadMetadataDetails[] loadDetails =
+    //        CarbonUtil.readLoadMetadata(loadModel.schema.getCarbonTable().getMetaDataFilepath());
+    writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(),
+        new ArrayList<LoadMetadataDetails>());
+
+    String segLocation =
+        storeLocation + "/" + databaseName + "/" + tableName + "/Fact/Part0/Segment_0";
+    File file = new File(segLocation);
+    File factFile = null;
+    File[] folderList = file.listFiles();
+    File folder = null;
+    for (int i = 0; i < folderList.length; i++) {
+      if (folderList[i].isDirectory()) {
+        folder = folderList[i];
+      }
+    }
+    if (folder.isDirectory()) {
+      File[] files = folder.listFiles();
+      for (int i = 0; i < files.length; i++) {
+        if (!files[i].isDirectory() && files[i].getName().startsWith("part")) {
+          factFile = files[i];
+          break;
+        }
+      }
+      //      Files.copy(factFile.toPath(), file.toPath(), REPLACE_EXISTING);
+      factFile.renameTo(new File(segLocation + "/" + factFile.getName()));
+      CarbonUtil.deleteFoldersAndFiles(folder);
+    }
+  }
+
+  public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databaseName,
+      String tableName, List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
+    LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
+    loadMetadataDetails.setLoadEndTime(System.currentTimeMillis());
+    loadMetadataDetails.setLoadStatus("SUCCESS");
+    loadMetadataDetails.setLoadName(String.valueOf(0));
+    loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
+    listOfLoadFolderDetails.add(loadMetadataDetails);
+
+    String dataLoadLocation = schema.getCarbonTable().getMetaDataFilepath() + File.separator
+        + CarbonCommonConstants.LOADMETADATA_FILENAME;
+
+    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) {
+        throw e;
+
+      }
+      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 void main(String[] args) {
+    StoreCreator.createCarbonStore();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/test/java/org/apache/carbondata/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/test/util/StoreCreator.java b/processing/src/test/java/org/apache/carbondata/test/util/StoreCreator.java
deleted file mode 100644
index 78e7e83..0000000
--- a/processing/src/test/java/org/apache/carbondata/test/util/StoreCreator.java
+++ /dev/null
@@ -1,539 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.test.util;
-
-import com.google.gson.Gson;
-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.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.converter.SchemaConverter;
-import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-import org.apache.carbondata.core.metadata.schema.SchemaEvolution;
-import org.apache.carbondata.core.metadata.schema.SchemaEvolutionEntry;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.TableInfo;
-import org.apache.carbondata.core.metadata.schema.table.TableSchema;
-import org.apache.carbondata.core.metadata.schema.table.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.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.processing.csvreaderstep.BlockDetails;
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
-import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
-import org.apache.carbondata.core.writer.ThriftWriter;
-import org.apache.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
-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.core.fileoperations.AtomicFileOperations;
-import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
-import org.apache.carbondata.core.fileoperations.FileWriteOperation;
-import org.apache.carbondata.processing.api.dataloader.DataLoadModel;
-import org.apache.carbondata.processing.api.dataloader.SchemaInfo;
-import org.apache.carbondata.processing.csvload.DataGraphExecuter;
-import org.apache.carbondata.processing.dataprocessor.DataProcessTaskStatus;
-import org.apache.carbondata.processing.dataprocessor.IDataProcessStatus;
-import org.apache.carbondata.processing.graphgenerator.GraphGenerator;
-import org.apache.carbondata.processing.graphgenerator.GraphGeneratorException;
-import org.apache.hadoop.fs.Path;
-
-import java.io.*;
-import java.nio.charset.Charset;
-import java.text.SimpleDateFormat;
-import java.util.*;
-
-/**
- * This class will create store file based on provided schema
- *
- */
-public class StoreCreator {
-
-  private static AbsoluteTableIdentifier absoluteTableIdentifier;
-
-  static {
-    try {
-      String storePath = new File("target/store").getCanonicalPath();
-      String dbName = "testdb";
-      String tableName = "testtable";
-      absoluteTableIdentifier =
-          new AbsoluteTableIdentifier(storePath, new CarbonTableIdentifier(dbName, tableName, UUID.randomUUID().toString()));
-    } catch (IOException ex) {
-
-    }
-  }
-
-  public static AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  /**
-   * Create store without any restructure
-   */
-  public static void createCarbonStore() {
-
-    try {
-
-      String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
-      File storeDir = new File(absoluteTableIdentifier.getStorePath());
-      CarbonUtil.deleteFoldersAndFiles(storeDir);
-      CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS,
-          absoluteTableIdentifier.getStorePath());
-
-      String kettleHomePath = "../processing/carbonplugins";
-      CarbonTable table = createTable();
-      writeDictionary(factFilePath, table);
-      CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
-      LoadModel loadModel = new LoadModel();
-      String partitionId = "0";
-      loadModel.setSchema(schema);
-      loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
-      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-      loadModel.setFactFilePath(factFilePath);
-      loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
-
-      executeGraph(loadModel, absoluteTableIdentifier.getStorePath(), kettleHomePath);
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-
-  }
-
-  private static CarbonTable createTable() throws IOException {
-    TableInfo tableInfo = new TableInfo();
-    tableInfo.setStorePath(absoluteTableIdentifier.getStorePath());
-    tableInfo.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
-    TableSchema tableSchema = new TableSchema();
-    tableSchema.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-    List<ColumnSchema> columnSchemas = new ArrayList<ColumnSchema>();
-    ArrayList<Encoding> encodings = new ArrayList<>();
-    encodings.add(Encoding.DICTIONARY);
-    ColumnSchema id = new ColumnSchema();
-    id.setColumnName("ID");
-    id.setColumnar(true);
-    id.setDataType(DataType.INT);
-    id.setEncodingList(encodings);
-    id.setColumnUniqueId(UUID.randomUUID().toString());
-    id.setDimensionColumn(true);
-    id.setColumnGroup(1);
-    columnSchemas.add(id);
-
-    ColumnSchema date = new ColumnSchema();
-    date.setColumnName("date");
-    date.setColumnar(true);
-    date.setDataType(DataType.STRING);
-    date.setEncodingList(encodings);
-    date.setColumnUniqueId(UUID.randomUUID().toString());
-    date.setDimensionColumn(true);
-    date.setColumnGroup(2);
-    columnSchemas.add(date);
-
-    ColumnSchema country = new ColumnSchema();
-    country.setColumnName("country");
-    country.setColumnar(true);
-    country.setDataType(DataType.STRING);
-    country.setEncodingList(encodings);
-    country.setColumnUniqueId(UUID.randomUUID().toString());
-    country.setDimensionColumn(true);
-    country.setColumnGroup(3);
-    columnSchemas.add(country);
-
-    ColumnSchema name = new ColumnSchema();
-    name.setColumnName("name");
-    name.setColumnar(true);
-    name.setDataType(DataType.STRING);
-    name.setEncodingList(encodings);
-    name.setColumnUniqueId(UUID.randomUUID().toString());
-    name.setDimensionColumn(true);
-    name.setColumnGroup(4);
-    columnSchemas.add(name);
-
-    ColumnSchema phonetype = new ColumnSchema();
-    phonetype.setColumnName("phonetype");
-    phonetype.setColumnar(true);
-    phonetype.setDataType(DataType.STRING);
-    phonetype.setEncodingList(encodings);
-    phonetype.setColumnUniqueId(UUID.randomUUID().toString());
-    phonetype.setDimensionColumn(true);
-    phonetype.setColumnGroup(5);
-    columnSchemas.add(phonetype);
-
-    ColumnSchema serialname = new ColumnSchema();
-    serialname.setColumnName("serialname");
-    serialname.setColumnar(true);
-    serialname.setDataType(DataType.STRING);
-    serialname.setEncodingList(encodings);
-    serialname.setColumnUniqueId(UUID.randomUUID().toString());
-    serialname.setDimensionColumn(true);
-    serialname.setColumnGroup(6);
-    columnSchemas.add(serialname);
-
-    ColumnSchema salary = new ColumnSchema();
-    salary.setColumnName("salary");
-    salary.setColumnar(true);
-    salary.setDataType(DataType.INT);
-    salary.setEncodingList(new ArrayList<Encoding>());
-    salary.setColumnUniqueId(UUID.randomUUID().toString());
-    salary.setDimensionColumn(false);
-    salary.setColumnGroup(7);
-    columnSchemas.add(salary);
-
-    tableSchema.setListOfColumns(columnSchemas);
-    SchemaEvolution schemaEvol = new SchemaEvolution();
-    schemaEvol.setSchemaEvolutionEntryList(new ArrayList<SchemaEvolutionEntry>());
-    tableSchema.setSchemaEvalution(schemaEvol);
-    tableSchema.setTableId(UUID.randomUUID().toString());
-    tableInfo.setTableUniqueName(
-        absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName() + "_"
-            + absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
-    tableInfo.setLastUpdatedTime(System.currentTimeMillis());
-    tableInfo.setFactTable(tableSchema);
-    tableInfo.setAggregateTableList(new ArrayList<TableSchema>());
-
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    String schemaFilePath = carbonTablePath.getSchemaFilePath();
-    String schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath);
-    tableInfo.setMetaDataFilepath(schemaMetadataPath);
-    CarbonMetadata.getInstance().loadTableMetadata(tableInfo);
-
-    SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
-    org.apache.carbondata.format.TableInfo thriftTableInfo = schemaConverter
-        .fromWrapperToExternalTableInfo(tableInfo, tableInfo.getDatabaseName(),
-            tableInfo.getFactTable().getTableName());
-    org.apache.carbondata.format.SchemaEvolutionEntry schemaEvolutionEntry =
-        new org.apache.carbondata.format.SchemaEvolutionEntry(tableInfo.getLastUpdatedTime());
-    thriftTableInfo.getFact_table().getSchema_evolution().getSchema_evolution_history()
-        .add(schemaEvolutionEntry);
-
-    FileFactory.FileType fileType = FileFactory.getFileType(schemaMetadataPath);
-    if (!FileFactory.isFileExist(schemaMetadataPath, fileType)) {
-      FileFactory.mkdirs(schemaMetadataPath, fileType);
-    }
-
-    ThriftWriter thriftWriter = new ThriftWriter(schemaFilePath, false);
-    thriftWriter.open();
-    thriftWriter.write(thriftTableInfo);
-    thriftWriter.close();
-    return CarbonMetadata.getInstance().getCarbonTable(tableInfo.getTableUniqueName());
-  }
-
-  private static void writeDictionary(String factFilePath, CarbonTable table) throws Exception {
-    BufferedReader reader = new BufferedReader(new FileReader(factFilePath));
-    String header = reader.readLine();
-    String[] split = header.split(",");
-    List<CarbonColumn> allCols = new ArrayList<CarbonColumn>();
-    List<CarbonDimension> dims = table.getDimensionByTableName(table.getFactTableName());
-    allCols.addAll(dims);
-    List<CarbonMeasure> msrs = table.getMeasureByTableName(table.getFactTableName());
-    allCols.addAll(msrs);
-    Set<String>[] set = new HashSet[dims.size()];
-    for (int i = 0; i < set.length; i++) {
-      set[i] = new HashSet<String>();
-    }
-    String line = reader.readLine();
-    while (line != null) {
-      String[] data = line.split(",");
-      for (int i = 0; i < set.length; i++) {
-        set[i].add(data[i]);
-      }
-      line = reader.readLine();
-    }
-
-    Cache dictCache = CacheProvider.getInstance()
-        .createCache(CacheType.REVERSE_DICTIONARY, absoluteTableIdentifier.getStorePath());
-    for (int i = 0; i < set.length; i++) {
-      ColumnIdentifier columnIdentifier = new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
-      CarbonDictionaryWriter writer =
-          new CarbonDictionaryWriterImpl(absoluteTableIdentifier.getStorePath(),
-              absoluteTableIdentifier.getCarbonTableIdentifier(), columnIdentifier);
-      for (String value : set[i]) {
-        writer.write(value);
-      }
-      writer.close();
-      writer.commit();
-      Dictionary dict = (Dictionary) dictCache.get(
-          new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier.getCarbonTableIdentifier(),
-        		  columnIdentifier, dims.get(i).getDataType()));
-      CarbonDictionarySortInfoPreparator preparator =
-          new CarbonDictionarySortInfoPreparator();
-      List<String> newDistinctValues = new ArrayList<String>();
-      CarbonDictionarySortInfo dictionarySortInfo =
-          preparator.getDictionarySortInfo(newDistinctValues, dict, dims.get(i).getDataType());
-      CarbonDictionarySortIndexWriter carbonDictionaryWriter =
-          new CarbonDictionarySortIndexWriterImpl(
-              absoluteTableIdentifier.getCarbonTableIdentifier(), columnIdentifier,
-              absoluteTableIdentifier.getStorePath());
-      try {
-        carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex());
-        carbonDictionaryWriter.writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted());
-      } finally {
-        carbonDictionaryWriter.close();
-      }
-    }
-    reader.close();
-  }
-
-  /**
-   * Execute graph which will further load data
-   *
-   * @param loadModel
-   * @param storeLocation
-   * @param kettleHomePath
-   * @throws Exception
-   */
-  public static void executeGraph(LoadModel loadModel, String storeLocation, String kettleHomePath)
-      throws Exception {
-    System.setProperty("KETTLE_HOME", kettleHomePath);
-    new File(storeLocation).mkdirs();
-    String outPutLoc = storeLocation + "/etl";
-    String databaseName = loadModel.getDatabaseName();
-    String tableName = loadModel.getTableName();
-    String tempLocationKey = databaseName + '_' + tableName + "_1";
-    CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation);
-    CarbonProperties.getInstance().addProperty("store_output_location", outPutLoc);
-    CarbonProperties.getInstance().addProperty("send.signal.load", "false");
-    CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true");
-    CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1");
-    CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true");
-    CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true");
-    CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true");
-    CarbonProperties.getInstance().addProperty("high.cardinality.value", "100000");
-    CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false");
-    CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000");
-
-    String fileNamePrefix = "";
-
-    String graphPath =
-        outPutLoc + File.separator + loadModel.getDatabaseName() + File.separator + tableName
-            + File.separator + 0 + File.separator + 1 + File.separator + tableName + ".ktr";
-    File path = new File(graphPath);
-    if (path.exists()) {
-      path.delete();
-    }
-
-    DataProcessTaskStatus dataProcessTaskStatus = new DataProcessTaskStatus(databaseName, tableName);
-    dataProcessTaskStatus.setCsvFilePath(loadModel.getFactFilePath());
-    SchemaInfo info = new SchemaInfo();
-    BlockDetails blockDetails = new BlockDetails(new Path(loadModel.getFactFilePath()),
-        0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"});
-    GraphGenerator.blockInfo.put("qwqwq", new BlockDetails[] { blockDetails });
-    dataProcessTaskStatus.setBlocksID("qwqwq");
-    dataProcessTaskStatus.setEscapeCharacter("\\");
-    dataProcessTaskStatus.setQuoteCharacter("\"");
-    dataProcessTaskStatus.setCommentCharacter("#");
-    dataProcessTaskStatus.setDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
-    info.setDatabaseName(databaseName);
-    info.setTableName(tableName);
-
-    generateGraph(dataProcessTaskStatus, info, loadModel.getTableName(), "0", loadModel.getSchema(), null,
-        loadModel.getLoadMetadataDetails());
-
-    DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus);
-    graphExecuter
-        .executeGraph(graphPath, info, loadModel.getSchema());
-    //    LoadMetadataDetails[] loadDetails =
-    //        CarbonUtil.readLoadMetadata(loadModel.schema.getCarbonTable().getMetaDataFilepath());
-    writeLoadMetadata(loadModel.schema, loadModel.getTableName(), loadModel.getTableName(),
-        new ArrayList<LoadMetadataDetails>());
-
-    String segLocation =
-        storeLocation + "/" + databaseName + "/" + tableName + "/Fact/Part0/Segment_0";
-    File file = new File(segLocation);
-    File factFile = null;
-    File[] folderList = file.listFiles();
-    File folder = null;
-    for (int i = 0; i < folderList.length; i++) {
-      if (folderList[i].isDirectory()) {
-        folder = folderList[i];
-      }
-    }
-    if (folder.isDirectory()) {
-      File[] files = folder.listFiles();
-      for (int i = 0; i < files.length; i++) {
-        if (!files[i].isDirectory() && files[i].getName().startsWith("part")) {
-          factFile = files[i];
-          break;
-        }
-      }
-      //      Files.copy(factFile.toPath(), file.toPath(), REPLACE_EXISTING);
-      factFile.renameTo(new File(segLocation + "/" + factFile.getName()));
-      CarbonUtil.deleteFoldersAndFiles(folder);
-    }
-  }
-
-  public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databaseName,
-      String tableName, List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
-    LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
-    loadMetadataDetails.setLoadEndTime(System.currentTimeMillis());
-    loadMetadataDetails.setLoadStatus("SUCCESS");
-    loadMetadataDetails.setLoadName(String.valueOf(0));
-    loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
-    listOfLoadFolderDetails.add(loadMetadataDetails);
-
-    String dataLoadLocation = schema.getCarbonTable().getMetaDataFilepath() + File.separator
-        + CarbonCommonConstants.LOADMETADATA_FILENAME;
-
-    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) {
-        throw e;
-
-      }
-      CarbonUtil.closeStreams(brWriter);
-
-    }
-    writeOperation.close();
-
-  }
-
-  /**
-   * generate graph
-   *
-   * @param dataProcessTaskStatus
-   * @param info
-   * @param tableName
-   * @param partitionID
-   * @param schema
-   * @param factStoreLocation
-   * @param loadMetadataDetails
-   * @throws GraphGeneratorException
-   */
-  private static void generateGraph(IDataProcessStatus dataProcessTaskStatus, SchemaInfo info,
-      String tableName, String partitionID, CarbonDataLoadSchema schema, String factStoreLocation,
-      List<LoadMetadataDetails> loadMetadataDetails)
-      throws GraphGeneratorException {
-    DataLoadModel model = new DataLoadModel();
-    model.setCsvLoad(null != dataProcessTaskStatus.getCsvFilePath() || null != dataProcessTaskStatus.getFilesToProcess());
-    model.setSchemaInfo(info);
-    model.setTableName(dataProcessTaskStatus.getTableName());
-    model.setTaskNo("1");
-    model.setBlocksID(dataProcessTaskStatus.getBlocksID());
-    model.setFactTimeStamp(System.currentTimeMillis() + "");
-    model.setEscapeCharacter(dataProcessTaskStatus.getEscapeCharacter());
-    model.setQuoteCharacter(dataProcessTaskStatus.getQuoteCharacter());
-    model.setCommentCharacter(dataProcessTaskStatus.getCommentCharacter());
-    model.setDateFormat(dataProcessTaskStatus.getDateFormat());
-    String outputLocation = CarbonProperties.getInstance()
-        .getProperty("store_output_location", "../carbon-store/system/carbon/etl");
-    GraphGenerator generator =
-        new GraphGenerator(model, partitionID, factStoreLocation, schema, "0", outputLocation);
-    generator.generateGraph();
-  }
-
-  public static String readCurrentTime() {
-    SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
-    String date = null;
-
-    date = sdf.format(new Date());
-
-    return date;
-  }
-
-  /**
-   * This is local model object used inside this class to store information related to data loading
-   */
-  private static class LoadModel {
-
-    private CarbonDataLoadSchema schema;
-    private String tableName;
-    private String databaseName;
-    private List<LoadMetadataDetails> loadMetaDetail;
-    private String factFilePath;
-
-    public void setSchema(CarbonDataLoadSchema schema) {
-      this.schema = schema;
-    }
-
-    public List<LoadMetadataDetails> getLoadMetadataDetails() {
-      return loadMetaDetail;
-    }
-
-    public CarbonDataLoadSchema getSchema() {
-      return schema;
-    }
-
-    public String getFactFilePath() {
-      return factFilePath;
-    }
-
-    public String getTableName() {
-      return tableName;
-    }
-
-    public String getDatabaseName() {
-      return databaseName;
-    }
-
-    public void setLoadMetadataDetails(List<LoadMetadataDetails> loadMetaDetail) {
-      this.loadMetaDetail = loadMetaDetail;
-    }
-
-    public void setFactFilePath(String factFilePath) {
-      this.factFilePath = factFilePath;
-    }
-
-    public void setTableName(String tableName) {
-      this.tableName = tableName;
-    }
-
-    public void setDatabaseName(String databaseName) {
-      this.databaseName = databaseName;
-    }
-
-  }
-
-  public static void main(String[] args) {
-    StoreCreator.createCarbonStore();
-  }
-
-}
\ No newline at end of file



[06/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index 6030a30..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepMeta.java
+++ /dev/null
@@ -1,470 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.step;
-
-import java.util.List;
-import java.util.Map;
-
-import org.pentaho.di.core.CheckResult;
-import org.pentaho.di.core.CheckResultInterface;
-import org.pentaho.di.core.Counter;
-import org.pentaho.di.core.database.DatabaseMeta;
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.exception.KettleXMLException;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.core.xml.XMLHandler;
-import org.pentaho.di.i18n.BaseMessages;
-import org.pentaho.di.repository.ObjectId;
-import org.pentaho.di.repository.Repository;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStepMeta;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-import org.w3c.dom.Node;
-
-public class CarbonSliceMergerStepMeta extends BaseStepMeta
-    implements StepMetaInterface, Cloneable {
-
-  /**
-   * for i18n purposes
-   */
-  private static final Class<?> PKG = CarbonSliceMergerStepMeta.class;
-
-  /**
-   * table name
-   */
-  private String tabelName;
-
-  /**
-   * mdkey size
-   */
-  private String mdkeySize;
-
-  /**
-   * measureCount
-   */
-  private String measureCount;
-
-  /**
-   * heirAndKeySize
-   */
-  private String heirAndKeySize;
-
-  /**
-   * databaseName
-   */
-  private String databaseName;
-
-  /**
-   * tableName
-   */
-  private String tableName;
-
-  /**
-   * isGroupByEnabled
-   */
-  private String groupByEnabled;
-
-  /**
-   * aggregatorString
-   */
-  private String aggregatorString;
-
-  /**
-   * aggregatorClassString
-   */
-  private String aggregatorClassString;
-
-  /**
-   * factDimLensString
-   */
-  private String factDimLensString;
-
-  private String levelAnddataTypeString;
-  /**
-   * partitionID
-   */
-  private String partitionID;
-  /**
-   * Id of the load folder
-   */
-  private String segmentId;
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-
-  /**
-   * CarbonDataWriterStepMeta constructor to initialize this class
-   */
-  public CarbonSliceMergerStepMeta() {
-    super();
-  }
-
-  /**
-   * set the default value for all the properties
-   */
-  @Override public void setDefault() {
-    tabelName = "";
-    mdkeySize = "";
-    measureCount = "";
-    heirAndKeySize = "";
-    tableName = "";
-    databaseName = "";
-    groupByEnabled = "";
-    aggregatorClassString = "";
-    aggregatorString = "";
-    factDimLensString = "";
-    levelAnddataTypeString = "";
-    partitionID = "";
-    segmentId = "";
-    taskNo = "";
-  }
-
-  /**
-   * Get the XML that represents the values in this step
-   *
-   * @return the XML that represents the metadata in this step
-   * @throws KettleException in case there is a conversion or XML encoding error
-   */
-  public String getXML() {
-    StringBuilder retval = new StringBuilder(150);
-    retval.append("    ").append(XMLHandler.addTagValue("TableName", tabelName));
-    retval.append("    ").append(XMLHandler.addTagValue("MDKeySize", mdkeySize));
-    retval.append("    ").append(XMLHandler.addTagValue("Measurecount", measureCount));
-    retval.append("    ").append(XMLHandler.addTagValue("HeirAndKeySize", heirAndKeySize));
-    retval.append("    ").append(XMLHandler.addTagValue("tableName", tableName));
-    retval.append("    ").append(XMLHandler.addTagValue("databaseName", databaseName));
-    retval.append("    ").append(XMLHandler.addTagValue("isGroupByEnabled", groupByEnabled));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("aggregatorClassString", aggregatorClassString));
-    retval.append("    ").append(XMLHandler.addTagValue("aggregatorString", aggregatorString));
-    retval.append("    ").append(XMLHandler.addTagValue("factDimLensString", factDimLensString));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("levelAnddataTypeString", levelAnddataTypeString));
-    retval.append("    ").append(XMLHandler.addTagValue("partitionID", partitionID));
-    retval.append("    ").append(XMLHandler.addTagValue("segmentId", segmentId));
-    retval.append("    ").append(XMLHandler.addTagValue("taskNo", taskNo));
-    return retval.toString();
-  }
-
-  /**
-   * Load the values for this step from an XML Node
-   *
-   * @param stepnode  the Node to get the info from
-   * @param databases The available list of databases to reference to
-   * @param counters  Counters to reference.
-   * @throws KettleXMLException When an unexpected XML error occurred. (malformed etc.)
-   */
-  @Override public void loadXML(Node stepnode, List<DatabaseMeta> databases,
-      Map<String, Counter> counters) throws KettleXMLException {
-    try {
-      databaseName = XMLHandler.getTagValue(stepnode, "databaseName");
-      tabelName = XMLHandler.getTagValue(stepnode, "TableName");
-      mdkeySize = XMLHandler.getTagValue(stepnode, "MDKeySize");
-      measureCount = XMLHandler.getTagValue(stepnode, "Measurecount");
-      heirAndKeySize = XMLHandler.getTagValue(stepnode, "HeirAndKeySize");
-      tableName = XMLHandler.getTagValue(stepnode, "tableName");
-      groupByEnabled = XMLHandler.getTagValue(stepnode, "isGroupByEnabled");
-      aggregatorClassString = XMLHandler.getTagValue(stepnode, "aggregatorClassString");
-      aggregatorString = XMLHandler.getTagValue(stepnode, "aggregatorString");
-      factDimLensString = XMLHandler.getTagValue(stepnode, "factDimLensString");
-      levelAnddataTypeString = XMLHandler.getTagValue(stepnode, "levelAnddataTypeString");
-      partitionID = XMLHandler.getTagValue(stepnode, "partitionID");
-      segmentId = XMLHandler.getTagValue(stepnode, "segmentId");
-      taskNo = XMLHandler.getTagValue(stepnode, "taskNo");
-    } catch (Exception e) {
-      throw new KettleXMLException("Unable to read step info from XML node", e);
-    }
-  }
-
-  /**
-   * Save the steps data into a Kettle repository
-   *
-   * @param rep              The Kettle repository to save to
-   * @param idTransformation The transformation ID
-   * @param idStep           The step ID
-   * @throws KettleException When an unexpected error occurred (database, network, etc)
-   */
-  @Override public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep)
-      throws KettleException {
-    try {
-      rep.saveStepAttribute(idTransformation, idStep, "TableName", tabelName); //$NON-NLS-1$
-      rep.saveStepAttribute(idTransformation, idStep, "MDKeySize", mdkeySize); //$NON-NLS-1$
-      rep.saveStepAttribute(idTransformation, idStep, "Measurecount", measureCount);
-      rep.saveStepAttribute(idTransformation, idStep, "HeirAndKeySize",
-          heirAndKeySize); //$NON-NLS-1$
-      rep.saveStepAttribute(idTransformation, idStep, "tableName", tableName); //$NON-NLS-1$
-      rep.saveStepAttribute(idTransformation, idStep, "databaseName", databaseName); //$NON-NLS-1$
-      rep.saveStepAttribute(idTransformation, idStep, "isGroupByEnabled", groupByEnabled);
-      rep.saveStepAttribute(idTransformation, idStep, "aggregatorClassString",
-          aggregatorClassString);
-      rep.saveStepAttribute(idTransformation, idStep, "aggregatorString", aggregatorString);
-      rep.saveStepAttribute(idTransformation, idStep, "factDimLensString", factDimLensString);
-      rep.saveStepAttribute(idTransformation, idStep, "levelAnddataTypeString",
-          levelAnddataTypeString);
-      rep.saveStepAttribute(idTransformation, idStep, "partitionID", partitionID);
-      rep.saveStepAttribute(idTransformation, idStep, "segmentId", segmentId);
-      rep.saveStepAttribute(idTransformation, idStep, "taskNo", taskNo);
-    } catch (Exception e) {
-      throw new KettleException(
-          BaseMessages.getString(PKG, "TemplateStep.Exception.UnableToSaveStepInfoToRepository")
-              + idStep, e);
-    }
-  }
-
-  /**
-   * Make an exact copy of this step, make sure to explicitly copy Collections
-   * etc.
-   *
-   * @return an exact copy of this step
-   */
-  public Object clone() {
-    Object retval = super.clone();
-    return retval;
-  }
-
-  /**
-   * Read the steps information from a Kettle repository
-   *
-   * @param rep       The repository to read from
-   * @param idStep    The step ID
-   * @param databases The databases to reference
-   * @param counters  The counters to reference
-   * @throws KettleException When an unexpected error occurred (database, network, etc)
-   */
-  @Override public void readRep(Repository rep, ObjectId idStep, List<DatabaseMeta> databases,
-      Map<String, Counter> counters) throws KettleException {
-    try {
-      tabelName = rep.getStepAttributeString(idStep, "TableName");
-      mdkeySize = rep.getStepAttributeString(idStep, "MDKeySize");
-      measureCount = rep.getStepAttributeString(idStep, "Measurecount");
-      heirAndKeySize = rep.getStepAttributeString(idStep, "HeirAndKeySize");
-      databaseName = rep.getStepAttributeString(idStep, "databaseName");
-      tableName = rep.getStepAttributeString(idStep, "tableName");
-      groupByEnabled = rep.getStepAttributeString(idStep, "isGroupByEnabled");
-      aggregatorClassString = rep.getStepAttributeString(idStep, "aggregatorClassString");
-      aggregatorString = rep.getStepAttributeString(idStep, "aggregatorString");
-      factDimLensString = rep.getStepAttributeString(idStep, "factDimLensString");
-      levelAnddataTypeString = rep.getStepAttributeString(idStep, "levelAnddataTypeString");
-      partitionID = rep.getStepAttributeString(idStep, "partitionID");
-      segmentId = rep.getStepAttributeString(idStep, "segmentId");
-      taskNo = rep.getStepAttributeString(idStep, "taskNo");
-    } catch (Exception exception) {
-      throw new KettleException(BaseMessages
-          .getString(PKG, "CarbonDataWriterStepMeta.Exception.UnexpectedErrorInReadingStepInfo"),
-          exception);
-    }
-
-  }
-
-  /**
-   * Get the executing step, needed by Trans to launch a step.
-   *
-   * @param stepMeta          The step info
-   * @param stepDataInterface the step data interface linked to this step. Here the step can
-   *                          store temporary data, database connections, etc.
-   * @param copyNr            The copy nr to get
-   * @param transMeta         The transformation info
-   * @param trans             The launching transformation
-   */
-  @Override public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface,
-      int copyNr, TransMeta transMeta, Trans trans) {
-    return new CarbonSliceMergerStep(stepMeta, stepDataInterface, copyNr, transMeta, trans);
-  }
-
-  /**
-   * Checks the settings of this step and puts the findings in a remarks List.
-   *
-   * @param remarks  The list to put the remarks in @see
-   *                 org.pentaho.di.core.CheckResult
-   * @param stepMeta The stepMeta to help checking
-   * @param prev     The fields coming from the previous step
-   * @param input    The input step names
-   * @param output   The output step names
-   * @param info     The fields that are used as information by the step
-   */
-  @Override public void check(List<CheckResultInterface> remarks, TransMeta transMeta,
-      StepMeta stepMeta, RowMetaInterface prev, String[] input, String[] output,
-      RowMetaInterface info) {
-
-    CheckResult checkResVal;
-
-    // See if we have input streams leading to this step!
-    if (input.length > 0) {
-      checkResVal =
-          new CheckResult(CheckResult.TYPE_RESULT_OK, "Step is receiving info from other steps.",
-              stepMeta);
-      remarks.add(checkResVal);
-    } else {
-      checkResVal =
-          new CheckResult(CheckResult.TYPE_RESULT_ERROR, "No input received from other steps!",
-              stepMeta);
-      remarks.add(checkResVal);
-    }
-
-  }
-
-  /**
-   * Get a new instance of the appropriate data class. This data class
-   * implements the StepDataInterface. It basically contains the persisting
-   * data that needs to live on, even if a worker thread is terminated.
-   *
-   * @return The appropriate StepDataInterface class.
-   */
-  @Override public StepDataInterface getStepData() {
-    return new CarbonSliceMergerStepData();
-  }
-
-  /**
-   * This method will return the table name
-   *
-   * @return tabelName
-   */
-  public String getTabelName() {
-    return tabelName;
-  }
-
-  /**
-   * This method will set the table name
-   *
-   * @param tabelName
-   */
-  public void setTabelName(String tabelName) {
-    this.tabelName = tabelName;
-  }
-
-  /**
-   * This method will be used to set the mdkey
-   *
-   * @param mdkeySize
-   */
-  public void setMdkeySize(String mdkeySize) {
-    this.mdkeySize = mdkeySize;
-  }
-
-  /**
-   * This method will be used to set the measure count
-   *
-   * @param measureCount
-   */
-  public void setMeasureCount(String measureCount) {
-    this.measureCount = measureCount;
-  }
-
-  /**
-   * This method will be used to set the heir and key size string
-   *
-   * @param heirAndKeySize
-   */
-  public void setHeirAndKeySize(String heirAndKeySize) {
-    this.heirAndKeySize = heirAndKeySize;
-  }
-
-  /**
-   * @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;
-  }
-
-  /**
-   * @param isGroupByEnabled the isGroupByEnabled to set
-   */
-  public void setGroupByEnabled(String isGroupByEnabled) {
-    this.groupByEnabled = isGroupByEnabled;
-  }
-
-  /**
-   * @param aggregatorString the aggregatorString to set
-   */
-  public void setAggregatorString(String aggregatorString) {
-    this.aggregatorString = aggregatorString;
-  }
-
-  /**
-   * @param aggregatorClassString the aggregatorClassString to set
-   */
-  public void setAggregatorClassString(String aggregatorClassString) {
-    this.aggregatorClassString = aggregatorClassString;
-  }
-
-  /**
-   * @param factDimLensString1 the factDimLensString to set
-   */
-  public void setFactDimLensString(String factDimLensString1) {
-    this.factDimLensString = factDimLensString1;
-  }
-
-  public void setLevelAnddataTypeString(String levelAnddataTypeString) {
-    this.levelAnddataTypeString = levelAnddataTypeString;
-  }
-
-  /**
-   * @param partitionID
-   */
-  public void setPartitionID(String partitionID) {
-    this.partitionID = partitionID;
-  }
-
-  /**
-   * set segment Id
-   * @param segmentId
-   */
-  public void setSegmentId(String segmentId) {
-    this.segmentId = segmentId;
-  }
-
-  /**
-   * @param taskNo
-   */
-  public void setTaskNo(String taskNo) {
-    this.taskNo = taskNo;
-  }
-
-  /**
-   * @return
-   */
-  public String getTaskNo() {
-    return taskNo;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 433f8db..ad96578 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
@@ -85,7 +85,7 @@ public class ParallelReadMergeSorterImpl implements Sorter {
             sortParameters.getDimColCount(),
             sortParameters.getComplexDimColCount(), sortParameters.getMeasureColCount(),
             sortParameters.getNoDictionaryCount(), sortParameters.getAggType(),
-            sortParameters.getNoDictionaryDimnesionColumn(), sortParameters.isUseKettle());
+            sortParameters.getNoDictionaryDimnesionColumn());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
index 813d83d..e3049d2 100644
--- 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
@@ -139,8 +139,7 @@ public class ParallelReadMergeSorterWithBucketingImpl implements Sorter {
         new SingleThreadFinalSortFilesMerger(dataFolderLocation, sortParameters.getTableName(),
             sortParameters.getDimColCount(), sortParameters.getComplexDimColCount(),
             sortParameters.getMeasureColCount(), sortParameters.getNoDictionaryCount(),
-            sortParameters.getAggType(), sortParameters.getNoDictionaryDimnesionColumn(),
-            sortParameters.isUseKettle());
+            sortParameters.getAggType(), sortParameters.getNoDictionaryDimnesionColumn());
     return finalMerger;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
index 7eacd08..f10e73a 100644
--- 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
@@ -134,7 +134,7 @@ public class DataWriterProcessorStepImpl extends AbstractDataLoadProcessorStep {
           "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());
+      throw new CarbonDataLoadingException("There is an unexpected error: " + e.getMessage(), e);
     }
     return null;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ArrayWrapper.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ArrayWrapper.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ArrayWrapper.java
deleted file mode 100644
index 27c3718..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ArrayWrapper.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.schema.metadata;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-public class ArrayWrapper implements Serializable {
-
-  /**
-   * Comment for <code>serialVersionUID</code>
-   */
-  private static final long serialVersionUID = -2016551342632572869L;
-
-  /**
-   * data
-   */
-  private int[] data;
-
-  public ArrayWrapper(int[] data) {
-    if (data == null) {
-      throw new IllegalArgumentException();
-    }
-    this.data = data;
-  }
-
-  @Override public boolean equals(Object other) {
-    if (other instanceof ArrayWrapper) {
-      return Arrays.equals(data, ((ArrayWrapper) other).data);
-    } else {
-      return false;
-    }
-
-  }
-
-  @Override public int hashCode() {
-    return Arrays.hashCode(data);
-  }
-
-  public int[] getData() {
-    return data;
-  }
-
-  public void setData(int[] data) {
-    this.data = data;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetails.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetails.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetails.java
deleted file mode 100644
index 2484002..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetails.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.schema.metadata;
-
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.util.DataTypeUtil;
-
-/**
- * Class holds the common column schema details needed for the data load
- */
-public class ColumnSchemaDetails {
-
-  /**
-   * column Name
-   */
-  private String columnName;
-  /**
-   * column datatype
-   */
-  private DataType columnType;
-  /**
-   * boolean to identify direct dictionary column
-   */
-  private Boolean isDirectDictionary;
-
-  /**
-   * Constructor to initialize object from the input string separated by comma (,)
-   *
-   * @param input
-   */
-  ColumnSchemaDetails(String input) {
-    String[] splits = input.split(",");
-    columnName = splits[0];
-    columnType = DataTypeUtil.getDataType(splits[1]);
-    isDirectDictionary = Boolean.parseBoolean(splits[2]);
-  }
-
-  /**
-   * Constructor to initialize the ColumnSchemaDetails
-   *
-   * @param columnName
-   * @param columnType
-   * @param isDirectDictionary
-   */
-  public ColumnSchemaDetails(String columnName, DataType columnType, Boolean isDirectDictionary) {
-    this.columnName = columnName;
-    this.columnType = columnType;
-    this.isDirectDictionary = isDirectDictionary;
-
-  }
-
-  /**
-   * returns the ColumnName
-   *
-   * @return
-   */
-  public String getColumnName() {
-    return columnName;
-  }
-
-  /**
-   * returns the dataType of the column
-   *
-   * @return
-   */
-  public DataType getColumnType() {
-    return columnType;
-  }
-
-  /**
-   * returns boolean value to identify direct dictionary
-   *
-   * @return
-   */
-  public Boolean isDirectDictionary() {
-    return isDirectDictionary;
-  }
-
-  /**
-   * @return
-   */
-  public String toString() {
-    return columnName + "," + columnType + "," + isDirectDictionary;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetailsWrapper.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetailsWrapper.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetailsWrapper.java
deleted file mode 100644
index 554dd06..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnSchemaDetailsWrapper.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-/**
- * Wrapper class to hold the columnschema details
- */
-public class ColumnSchemaDetailsWrapper {
-
-  /**
-   * Map of the ColumnSchemaDetails
-   */
-  private Map<String, ColumnSchemaDetails> columnSchemaDetailsMap;
-
-  /**
-   * return the string object
-   *
-   * @return
-   */
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    Set<Map.Entry<String, ColumnSchemaDetails>> entries = columnSchemaDetailsMap.entrySet();
-    Iterator<Map.Entry<String, ColumnSchemaDetails>> iterator = entries.iterator();
-
-    while (iterator.hasNext()) {
-      Map.Entry<String, ColumnSchemaDetails>  entry = iterator.next();
-      builder.append(entry.getKey());
-      builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      builder.append(entry.getValue().toString());
-      if (iterator.hasNext()) {
-        builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      }
-    }
-    return builder.toString();
-  }
-
-  /**
-   * default constructor
-   */
-  public ColumnSchemaDetailsWrapper() {
-
-  }
-
-  /**
-   * Constructor take serialized string as input and populates the List of columnschema details
-   *
-   * @param input
-   */
-  public ColumnSchemaDetailsWrapper(String input) {
-    columnSchemaDetailsMap = new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    String[] split = input.split(CarbonCommonConstants.HASH_SPC_CHARACTER);
-    for (int i = 0; i < split.length; i++) {
-      String key = split[i++];
-      ColumnSchemaDetails details = new ColumnSchemaDetails(split[i]);
-      columnSchemaDetailsMap.put(key, details);
-    }
-  }
-
-  /**
-   * returns ColumnSchemaDetails of all columns
-   *
-   * @return
-   */
-  public Map<String, ColumnSchemaDetails> getColumnSchemaDetailsMap() {
-    return columnSchemaDetailsMap;
-  }
-
-  /**
-   * sets the map of column schema
-   *
-   * @param columnSchemaDetailsMap
-   */
-  public void setColumnSchemaDetailsMap(Map<String, ColumnSchemaDetails> columnSchemaDetailsMap) {
-    this.columnSchemaDetailsMap = columnSchemaDetailsMap;
-  }
-
-  /**
-   * returns the columnSchemaDetails of requested column
-   *
-   * @param key
-   * @return
-   */
-  public ColumnSchemaDetails get(String key) {
-    return columnSchemaDetailsMap.get(key);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index b8b8bcd..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/ColumnsInfo.java
+++ /dev/null
@@ -1,430 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.processing.datatypes.GenericDataType;
-
-public class ColumnsInfo {
-
-  /**
-   * Indices for dimensions in the record. Doesn't include any properties.
-   */
-  private int[] dims;
-
-  /**
-   * Map<HierTableName, KeyGenerator>
-   */
-  private Map<String, KeyGenerator> keyGenerators;
-
-  /**
-   * Hierarchy table names (Same will be file names for file store or
-   * incremental load)
-   */
-  private Set<String> hierTables;
-
-  /**
-   * Batch size configured in transformation
-   */
-  private int batchSize;
-
-  /**
-   * To decide it is data load for aggregate table or not.
-   */
-  private boolean isAggregateLoad;
-
-  /**
-   * Store type DB or file based ?
-   */
-  private String storeType;
-
-  /**
-   * column Names for dimensions. Which will be used as table name for store
-   */
-  private String[] dimColNames;
-
-  /**
-   * baseStoreLocation
-   */
-  private String baseStoreLocation;
-
-  /**
-   * Maximum possible surrogate key for dimension possible based on
-   * cardinality value in schema definition
-   */
-  private int[] maxKeys;
-
-  /**
-   * Dimension Index, Properties indices in the tuple.
-   * [0] - [2,3] - 2 Props at indices 2 & 3 [1] - [4,7,8] - 3 props at indices
-   * 4,7, & 8 [2] - [] - No props
-   */
-  private int[][] propIndx;
-
-  /**
-   * Dimension Index, Property column names from table.
-   * [0] - [col2,col3] [1] - [col4,col7,col8] [2] - []
-   */
-  private List<String>[] propColumns;
-
-  /**
-   * timDimIndex
-   */
-  private int timDimIndex;
-
-  /**
-   * timDimIndexEnd
-   */
-  private int timDimIndexEnd;
-
-  /**
-   * timeOrdinalIndices
-   */
-  private int[] timeOrdinalIndices;
-
-  /**
-   * timeOrdinalCols
-   */
-  private String[] timeOrdinalCols;
-
-  /**
-   * propTypes
-   */
-  private List<String>[] propTypes;
-
-  /**
-   * dimHierRel
-   */
-  private String[] dimHierRel;
-
-  /**
-   * tableName
-   */
-  private String tableName;
-
-  /**
-   * Primary key Map
-   */
-  private Map<String, Boolean> primaryKeyMap;
-
-  /**
-   * measureColumns
-   */
-  private String[] measureColumns;
-
-  private boolean[] dimsPresent;
-
-  private String databaseName;
-
-  private Map<String, GenericDataType> complexTypesMap;
-
-  /**
-   * column Ids of dimensions in a table
-   */
-  private String[] dimensionColumnIds;
-
-  /**
-   * wrapper object having the columnSchemaDetails
-   */
-  private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper;
-
-  private Map<String, Map<String, String>> columnProperties;
-
-  public Map<String, GenericDataType> getComplexTypesMap() {
-    return complexTypesMap;
-  }
-
-  public void setComplexTypesMap(Map<String, GenericDataType> complexTypesMap) {
-    this.complexTypesMap = complexTypesMap;
-  }
-
-  /**
-   * @return Returns the dims.
-   */
-  public int[] getDims() {
-    return dims;
-  }
-
-  /**
-   * @param dims The dims to set.
-   */
-  public void setDims(int[] dims) {
-    this.dims = dims;
-  }
-
-  /**
-   * @return Returns the keyGenerators.
-   */
-  public Map<String, KeyGenerator> getKeyGenerators() {
-    return keyGenerators;
-  }
-
-  /**
-   * @param keyGenerators The keyGenerators to set.
-   */
-  public void setKeyGenerators(Map<String, KeyGenerator> keyGenerators) {
-    this.keyGenerators = keyGenerators;
-  }
-
-  /**
-   * @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 hierTables.
-   */
-  public Set<String> getHierTables() {
-    return hierTables;
-  }
-
-  /**
-   * @param hierTables The hierTables to set.
-   */
-  public void setHierTables(Set<String> hierTables) {
-    this.hierTables = hierTables;
-  }
-
-  /**
-   * @param batchSize The batchSize to set.
-   */
-  public void setBatchSize(int batchSize) {
-    this.batchSize = batchSize;
-  }
-
-  public void setAggregateLoad(boolean isAggregateLoad) {
-    this.isAggregateLoad = isAggregateLoad;
-  }
-
-  /**
-   * @param storeType The storeType to set.
-   */
-  public void setStoreType(String storeType) {
-    this.storeType = storeType;
-  }
-
-  /**
-   * @return Returns the dimColNames.
-   */
-  public String[] getDimColNames() {
-    return dimColNames;
-  }
-
-  /**
-   * @param dimColNames The dimColNames to set.
-   */
-  public void setDimColNames(String[] dimColNames) {
-    this.dimColNames = dimColNames;
-  }
-
-  /**
-   * @return Returns the maxKeys.
-   */
-  public int[] getMaxKeys() {
-    return maxKeys;
-  }
-
-  /**
-   * @param maxKeys The maxKeys to set.
-   */
-  public void setMaxKeys(int[] maxKeys) {
-    this.maxKeys = maxKeys;
-  }
-
-  /**
-   * @return Returns the propIndx.
-   */
-  public int[][] getPropIndx() {
-    return propIndx;
-  }
-
-  /**
-   * @param propIndx The propIndx to set.
-   */
-  public void setPropIndx(int[][] propIndx) {
-    this.propIndx = propIndx;
-  }
-
-  /**
-   * @param propColumns The propColumns to set.
-   */
-  public void setPropColumns(List<String>[] propColumns) {
-    this.propColumns = propColumns;
-  }
-
-  /**
-   * @return Returns the timDimIndex.
-   */
-  public int getTimDimIndex() {
-    return timDimIndex;
-  }
-
-  /**
-   * @param timDimIndex The timDimIndex to set.
-   */
-  public void setTimDimIndex(int timDimIndex) {
-    this.timDimIndex = timDimIndex;
-  }
-
-  /**
-   * @return Returns the timDimIndexEnd.
-   */
-  public int getTimDimIndexEnd() {
-    return timDimIndexEnd;
-  }
-
-  /**
-   * @return Returns the timeOrdinalIndices.
-   */
-  public int[] getTimeOrdinalIndices() {
-    return timeOrdinalIndices;
-  }
-
-  /**
-   * @param timeOrdinalIndices The timeOrdinalIndices to set.
-   */
-  public void setTimeOrdinalIndices(int[] timeOrdinalIndices) {
-    this.timeOrdinalIndices = timeOrdinalIndices;
-  }
-
-  /**
-   * @param timeOrdinalCols The timeOrdinalCols to set.
-   */
-  public void setTimeOrdinalCols(String[] timeOrdinalCols) {
-    this.timeOrdinalCols = timeOrdinalCols;
-  }
-
-  /**
-   * @param propTypes The propTypes to set.
-   */
-  public void setPropTypes(List<String>[] propTypes) {
-    this.propTypes = propTypes;
-  }
-
-  /**
-   * @return Returns the baseStoreLocation.
-   */
-  public String getBaseStoreLocation() {
-    return baseStoreLocation;
-  }
-
-  /**
-   * @param baseStoreLocation The baseStoreLocation to set.
-   */
-  public void setBaseStoreLocation(String baseStoreLocation) {
-    this.baseStoreLocation = baseStoreLocation;
-  }
-
-  /**
-   * @param dimHierRel The dimHierRel to set.
-   */
-  public void setDimHierRel(String[] dimHierRel) {
-    this.dimHierRel = dimHierRel;
-  }
-
-  /**
-   * @return Returns the primaryKeyMap.
-   */
-  public Map<String, Boolean> getPrimaryKeyMap() {
-    return primaryKeyMap;
-  }
-
-  /**
-   * @param primaryKeyMap The primaryKeyMap to set.
-   */
-  public void setPrimaryKeyMap(Map<String, Boolean> primaryKeyMap) {
-    this.primaryKeyMap = primaryKeyMap;
-  }
-
-  /**
-   * getDimsPresent
-   *
-   * @return boolean[]
-   */
-  public boolean[] getDimsPresent() {
-    return dimsPresent;
-  }
-
-  /**
-   * @param measureColumns The measureColumns to set.
-   */
-  public void setMeasureColumns(String[] measureColumns) {
-    this.measureColumns = measureColumns;
-  }
-
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * @return column Ids
-   */
-  public String[] getDimensionColumnIds() {
-    return dimensionColumnIds;
-  }
-
-  /**
-   * @param dimensionColumnIds column Ids for dimensions in a table
-   */
-  public void setDimensionColumnIds(String[] dimensionColumnIds) {
-    this.dimensionColumnIds = dimensionColumnIds;
-  }
-
-  /**
-   * returns wrapper object having the columnSchemaDetails
-   *
-   * @return
-   */
-  public ColumnSchemaDetailsWrapper getColumnSchemaDetailsWrapper() {
-    return columnSchemaDetailsWrapper;
-  }
-
-  /**
-   * set the wrapper object having the columnSchemaDetails
-   *
-   * @param columnSchemaDetailsWrapper
-   */
-  public void setColumnSchemaDetailsWrapper(ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper) {
-    this.columnSchemaDetailsWrapper = columnSchemaDetailsWrapper;
-  }
-
-  public void setColumnProperties(Map<String, Map<String, String>> columnProperties) {
-    this.columnProperties = columnProperties;
-  }
-
-  public Map<String, String> getColumnProperties(String columnName) {
-    return this.columnProperties.get(columnName);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index 32f096d..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/HierarchiesInfo.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.schema.metadata;
-
-import java.util.Map;
-
-public class HierarchiesInfo {
-
-  /**
-   * hierarichieName
-   */
-  private String hierarichieName;
-
-  /**
-   * columnPropMap
-   */
-  private Map<String, String[]> columnPropMap;
-
-  public String getHierarichieName() {
-    return hierarichieName;
-  }
-
-  public void setHierarichieName(String hierarichieName) {
-    this.hierarichieName = hierarichieName;
-  }
-
-  public Map<String, String[]> getColumnPropMap() {
-    return columnPropMap;
-  }
-
-  public void setColumnPropMap(Map<String, String[]> columnPropMap) {
-    this.columnPropMap = columnPropMap;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOption.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOption.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOption.java
deleted file mode 100644
index c01d800..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOption.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.schema.metadata;
-
-/**
- * This class is to hold the key value pair of properties needed while dataload.
- */
-public class TableOption {
-  /**
-   * option key name
-   */
-  private String optionKey;
-  /**
-   * option key value
-   */
-  private String optionValue;
-
-  /**
-   * the constructor to initialize the key value pair TableOption instance
-   *
-   * @param optionKey
-   * @param optionValue
-   */
-  public TableOption(String optionKey, String optionValue) {
-    this.optionKey = optionKey;
-    this.optionValue = optionValue;
-  }
-
-  /**
-   * constructor to init from te string separated by comma(,)
-   *
-   * @param str
-   */
-  public TableOption(String str) {
-    //passing 2 to split the key value pair having empty value for the corresponding key.
-    String[] split = str.split(",", 2);
-    this.optionKey = split[0];
-    this.optionValue = split[1];
-  }
-
-  /**
-   * returns options key
-   *
-   * @return
-   */
-  public String getOptionKey() {
-    return optionKey;
-  }
-
-  /**
-   * returns options value
-   *
-   * @return
-   */
-  public String getOptionValue() {
-    return optionValue;
-  }
-
-  /**
-   * @return
-   */
-  public String toString() {
-    return optionKey + "," + optionValue;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOptionWrapper.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOptionWrapper.java b/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOptionWrapper.java
deleted file mode 100644
index 02f797a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/schema/metadata/TableOptionWrapper.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-/**
- * The class hold the table option details being used while dataload
- */
-public class TableOptionWrapper {
-  /**
-   * map holds the table options
-   */
-  private static final Map<String, TableOption> mapOFOptions =
-      new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  private static TableOptionWrapper tableOptionWrapper = new TableOptionWrapper();
-
-  /**
-   * to  initialize the wrapper object
-   */
-  private TableOptionWrapper() {
-  }
-
-  /**
-   * @param input
-   */
-  public static void populateTableOptions(String input) {
-    String[] split =
-        null != input ? input.split(CarbonCommonConstants.HASH_SPC_CHARACTER) : new String[0];
-    for (String str : split) {
-      TableOption tableOption = new TableOption(str);
-      mapOFOptions.put(tableOption.getOptionKey(), tableOption);
-    }
-  }
-
-  /**
-   * @param input
-   */
-  public static void setTableOption(String input) {
-    if (null != input) {
-      TableOption tableOption = new TableOption(input);
-      mapOFOptions.put(tableOption.getOptionKey(), tableOption);
-    }
-  }
-
-  /**
-   * returns TableOptionWrapper instance
-   *
-   * @return
-   */
-  public static TableOptionWrapper getTableOptionWrapperInstance() {
-    return tableOptionWrapper;
-  }
-
-  /**
-   * returns the options key value
-   * return null if the key is not found in the map
-   *
-   * @param key
-   * @return
-   */
-  public String get(String key) {
-    TableOption tableOption = mapOFOptions.get(key);
-    return null != tableOption ? tableOption.getOptionValue() : null;
-  }
-
-  /**
-   * return the string object
-   *
-   * @return
-   */
-  public String toString() {
-    StringBuilder builder = new StringBuilder();
-    Set<Map.Entry<String, TableOption>> entries = mapOFOptions.entrySet();
-    Iterator<Map.Entry<String, TableOption>> iterator = entries.iterator();
-
-    while (iterator.hasNext()) {
-      Map.Entry<String, TableOption> entry = iterator.next();
-      builder.append(entry.getValue().toString());
-      builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-    }
-    return builder.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
index 1b16675..0ac2d5c 100644
--- 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
@@ -87,8 +87,6 @@ public class IntermediateFileMerger implements Callable<Void> {
 
   private File outPutFile;
 
-  private boolean useKettle;
-
   private boolean[] noDictionarycolumnMapping;
 
   /**
@@ -100,7 +98,6 @@ public class IntermediateFileMerger implements Callable<Void> {
     this.fileCounter = intermediateFiles.length;
     this.intermediateFiles = intermediateFiles;
     this.outPutFile = outPutFile;
-    this.useKettle = mergerParameters.isUseKettle();
     noDictionarycolumnMapping = mergerParameters.getNoDictionaryDimnesionColumn();
   }
 
@@ -111,14 +108,8 @@ public class IntermediateFileMerger implements Callable<Void> {
     try {
       startSorting();
       initialize();
-      if (useKettle) {
-        while (hasNext()) {
-          writeDataTofile(next());
-        }
-      } else {
-        while (hasNext()) {
-          writeDataTofileWithOutKettle(next());
-        }
+      while (hasNext()) {
+        writeDataTofile(next());
       }
       if (mergerParameters.isSortFileCompressionEnabled() || mergerParameters.isPrefetch()) {
         if (entryCount > 0) {
@@ -260,8 +251,7 @@ public class IntermediateFileMerger implements Callable<Void> {
           new SortTempFileChunkHolder(tempFile, mergerParameters.getDimColCount(),
               mergerParameters.getComplexDimColCount(), mergerParameters.getMeasureColCount(),
               mergerParameters.getFileBufferSize(), mergerParameters.getNoDictionaryCount(),
-              mergerParameters.getAggType(), mergerParameters.getNoDictionaryDimnesionColumn(),
-              mergerParameters.isUseKettle());
+              mergerParameters.getAggType(), mergerParameters.getNoDictionaryDimnesionColumn());
 
       // initialize
       sortTempFileChunkHolder.initialize();
@@ -283,7 +273,7 @@ public class IntermediateFileMerger implements Callable<Void> {
    */
   private void createRecordHolderQueue(File[] listFiles) {
     // creating record holder heap
-    this.recordHolderHeap = new PriorityQueue<SortTempFileChunkHolder>(listFiles.length);
+    this.recordHolderHeap = new PriorityQueue<>(listFiles.length);
   }
 
   /**
@@ -309,8 +299,6 @@ public class IntermediateFileMerger implements Callable<Void> {
   /**
    * Below method will be used to write data to file
    *
-   * TODO Remove it after kettle is removed
-   *
    * @throws CarbonSortKeyAndGroupByException problem while writing
    */
   private void writeDataTofile(Object[] row) throws CarbonSortKeyAndGroupByException {
@@ -330,71 +318,6 @@ public class IntermediateFileMerger implements Callable<Void> {
       return;
     }
     try {
-      int fieldIndex = 0;
-      char[] aggType = mergerParameters.getAggType();
-
-      for (int counter = 0; counter < mergerParameters.getDimColCount(); counter++) {
-        stream.writeInt((Integer) NonDictionaryUtil.getDimension(fieldIndex++, row));
-      }
-
-      // added for high card also
-      if ((mergerParameters.getNoDictionaryCount() + mergerParameters
-          .getComplexDimColCount()) > 0) {
-        stream.write(NonDictionaryUtil.getByteArrayForNoDictionaryCols(row));
-      }
-
-      fieldIndex = 0;
-      for (int counter = 0; counter < mergerParameters.getMeasureColCount(); counter++) {
-        if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
-          stream.write((byte) 1);
-          if (aggType[counter] == CarbonCommonConstants.BYTE_VALUE_MEASURE) {
-            Double val = (Double) NonDictionaryUtil.getMeasure(fieldIndex, row);
-            stream.writeDouble(val);
-          } else if (aggType[counter] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) {
-            Double val = (Double) NonDictionaryUtil.getMeasure(fieldIndex, row);
-            stream.writeDouble(val);
-          } else if (aggType[counter] == CarbonCommonConstants.BIG_INT_MEASURE) {
-            Long val = (Long) NonDictionaryUtil.getMeasure(fieldIndex, row);
-            stream.writeLong(val);
-          } else if (aggType[counter] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) {
-            byte[] bigDecimalInBytes = (byte[]) NonDictionaryUtil.getMeasure(fieldIndex, row);
-            stream.writeInt(bigDecimalInBytes.length);
-            stream.write(bigDecimalInBytes);
-          }
-        } else {
-          stream.write((byte) 0);
-        }
-
-        fieldIndex++;
-      }
-
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
-    }
-  }
-
-  /**
-   * Below method will be used to write data to file
-   *
-   * @throws CarbonSortKeyAndGroupByException problem while writing
-   */
-  private void writeDataTofileWithOutKettle(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 {
       char[] aggType = mergerParameters.getAggType();
       int[] mdkArray = (int[]) row[0];
       byte[][] nonDictArray = (byte[][]) row[1];

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
index 794935d..9b5a850 100644
--- 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
@@ -38,7 +38,6 @@ 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;
-import org.apache.carbondata.processing.util.NonDictionaryUtil;
 
 public class SortDataRows {
   /**
@@ -193,19 +192,10 @@ public class SortDataRows {
       toSort = new Object[entryCount][];
       System.arraycopy(recordHolderList, 0, toSort, 0, entryCount);
 
-      if (parameters.isUseKettle()) {
-        if (parameters.getNoDictionaryCount() > 0) {
-          Arrays.sort(toSort, new RowComparator(parameters.getNoDictionaryDimnesionColumn(),
-              parameters.getNoDictionaryCount()));
-        } else {
-          Arrays.sort(toSort, new RowComparatorForNormalDims(parameters.getDimColCount()));
-        }
+      if (parameters.getNoDictionaryCount() > 0) {
+        Arrays.sort(toSort, new NewRowComparator(parameters.getNoDictionaryDimnesionColumn()));
       } else {
-        if (parameters.getNoDictionaryCount() > 0) {
-          Arrays.sort(toSort, new NewRowComparator(parameters.getNoDictionaryDimnesionColumn()));
-        } else {
-          Arrays.sort(toSort, new NewRowComparatorForNormalDims(parameters.getDimColCount()));
-        }
+        Arrays.sort(toSort, new NewRowComparatorForNormalDims(parameters.getDimColCount()));
       }
       recordHolderList = toSort;
 
@@ -233,11 +223,7 @@ public class SortDataRows {
       writeSortTempFile(recordHolderList, entryCountLocal, file);
       return;
     }
-    if (parameters.isUseKettle()) {
-      writeData(recordHolderList, entryCountLocal, file);
-    } else {
-      writeDataWithOutKettle(recordHolderList, entryCountLocal, file);
-    }
+    writeData(recordHolderList, entryCountLocal, file);
   }
 
   private void writeSortTempFile(Object[][] recordHolderList, int entryCountLocal, File file)
@@ -258,7 +244,6 @@ public class SortDataRows {
     }
   }
 
-  // TODO Remove it after kettle got removed
   private void writeData(Object[][] recordHolderList, int entryCountLocal, File file)
       throws CarbonSortKeyAndGroupByException {
     DataOutputStream stream = null;
@@ -269,66 +254,6 @@ public class SortDataRows {
 
       // write number of entries to the file
       stream.writeInt(entryCountLocal);
-      int dimColCount = parameters.getDimColCount();
-      int combinedDimCount = parameters.getNoDictionaryCount() + parameters.getComplexDimColCount();
-      char[] aggType = parameters.getAggType();
-      Object[] row = null;
-      for (int i = 0; i < entryCountLocal; i++) {
-        // get row from record holder list
-        row = recordHolderList[i];
-        int fieldIndex = 0;
-
-        for (int dimCount = 0; dimCount < dimColCount; dimCount++) {
-          stream.writeInt(NonDictionaryUtil.getDimension(fieldIndex++, row));
-        }
-
-        // if any high cardinality dims are present then write it to the file.
-
-        if (combinedDimCount > 0) {
-          stream.write(NonDictionaryUtil.getByteArrayForNoDictionaryCols(row));
-        }
-
-        // as measures are stored in separate array.
-        fieldIndex = 0;
-        for (int mesCount = 0; mesCount < parameters.getMeasureColCount(); mesCount++) {
-          if (null != NonDictionaryUtil.getMeasure(fieldIndex, row)) {
-            stream.write((byte) 1);
-            if (aggType[mesCount] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) {
-              Double val = (Double) NonDictionaryUtil.getMeasure(fieldIndex, row);
-              stream.writeDouble(val);
-            } else if (aggType[mesCount] == CarbonCommonConstants.BIG_INT_MEASURE) {
-              Long val = (Long) NonDictionaryUtil.getMeasure(fieldIndex, row);
-              stream.writeLong(val);
-            } else if (aggType[mesCount] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) {
-              BigDecimal val = (BigDecimal) NonDictionaryUtil.getMeasure(fieldIndex, row);
-              byte[] bigDecimalInBytes = DataTypeUtil.bigDecimalToByte(val);
-              stream.writeInt(bigDecimalInBytes.length);
-              stream.write(bigDecimalInBytes);
-            }
-          } else {
-            stream.write((byte) 0);
-          }
-          fieldIndex++;
-        }
-      }
-    } catch (IOException e) {
-      throw new CarbonSortKeyAndGroupByException("Problem while writing the file", e);
-    } finally {
-      // close streams
-      CarbonUtil.closeStreams(stream);
-    }
-  }
-
-  private void writeDataWithOutKettle(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;
       char[] aggType = parameters.getAggType();
@@ -460,23 +385,12 @@ public class SortDataRows {
     @Override public Void call() throws Exception {
       try {
         long startTime = System.currentTimeMillis();
-        if (parameters.isUseKettle()) {
-          if (parameters.getNoDictionaryCount() > 0) {
-            Arrays.sort(recordHolderArray,
-                new RowComparator(parameters.getNoDictionaryDimnesionColumn(),
-                    parameters.getNoDictionaryCount()));
-          } else {
-            Arrays.sort(recordHolderArray,
-                new RowComparatorForNormalDims(parameters.getDimColCount()));
-          }
+        if (parameters.getNoDictionaryCount() > 0) {
+          Arrays.sort(recordHolderArray,
+              new NewRowComparator(parameters.getNoDictionaryDimnesionColumn()));
         } else {
-          if (parameters.getNoDictionaryCount() > 0) {
-            Arrays.sort(recordHolderArray,
-                new NewRowComparator(parameters.getNoDictionaryDimnesionColumn()));
-          } else {
-            Arrays.sort(recordHolderArray,
-                new NewRowComparatorForNormalDims(parameters.getDimColCount()));
-          }
+          Arrays.sort(recordHolderArray,
+              new NewRowComparatorForNormalDims(parameters.getDimColCount()));
         }
 
         // create a new file every time

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
index a52ebb2..d42dc32 100644
--- 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
@@ -114,11 +114,6 @@ public class SortParameters {
 
   private int numberOfCores;
 
-  /**
-   * TODO Temporary conf , it will be removed after kettle removal.
-   */
-  private boolean useKettle = true;
-
   public SortParameters getCopy() {
     SortParameters parameters = new SortParameters();
     parameters.tempFileLocation = tempFileLocation;
@@ -143,7 +138,6 @@ public class SortParameters {
     parameters.taskNo = taskNo;
     parameters.noDictionaryDimnesionColumn = noDictionaryDimnesionColumn;
     parameters.numberOfCores = numberOfCores;
-    parameters.useKettle = useKettle;
     return parameters;
   }
 
@@ -323,14 +317,6 @@ public class SortParameters {
     this.numberOfCores = numberOfCores;
   }
 
-  public boolean isUseKettle() {
-    return useKettle;
-  }
-
-  public void setUseKettle(boolean useKettle) {
-    this.useKettle = useKettle;
-  }
-
   public static SortParameters createSortParameters(CarbonDataLoadConfiguration configuration) {
     SortParameters parameters = new SortParameters();
     CarbonTableIdentifier tableIdentifier =
@@ -432,7 +418,6 @@ public class SortParameters {
     char[] aggType = CarbonDataProcessorUtil
         .getAggType(configuration.getMeasureCount(), configuration.getMeasureFields());
     parameters.setAggType(aggType);
-    parameters.setUseKettle(false);
     return parameters;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
index fef8c9d..ae01404 100644
--- 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
@@ -23,7 +23,6 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -32,7 +31,6 @@ 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.constants.IgnoreDictionary;
 import org.apache.carbondata.core.util.ByteUtil.UnsafeComparer;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
@@ -134,9 +132,6 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
    */
   private boolean[] isNoDictionaryDimensionColumn;
 
-  // TODO temporary configuration, remove after kettle removal
-  private boolean useKettle;
-
   /**
    * Constructor to initialize
    *
@@ -151,7 +146,7 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
    */
   public SortTempFileChunkHolder(File tempFile, int dimensionCount, int complexDimensionCount,
       int measureCount, int fileBufferSize, int noDictionaryCount, char[] aggType,
-      boolean[] isNoDictionaryDimensionColumn, boolean useKettle) {
+      boolean[] isNoDictionaryDimensionColumn) {
     // set temp file
     this.tempFile = tempFile;
 
@@ -166,7 +161,6 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
     this.executorService = Executors.newFixedThreadPool(1);
     this.aggType = aggType;
     this.isNoDictionaryDimensionColumn = isNoDictionaryDimensionColumn;
-    this.useKettle = useKettle;
   }
 
   /**
@@ -301,83 +295,11 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   }
 
   /**
-   * @return
-   * @throws CarbonSortKeyAndGroupByException
-   */
-  private Object[] getRowFromStream() throws CarbonSortKeyAndGroupByException {
-    // create new row of size 3 (1 for dims , 1 for high card , 1 for measures)
-    if (useKettle) {
-      return getRowFromStreamWithKettle();
-    } else {
-      return getRowFromStreamWithOutKettle();
-    }
-  }
-
-  // TODO remove after kettle flow is removed
-  private Object[] getRowFromStreamWithKettle() throws CarbonSortKeyAndGroupByException {
-    Object[] holder = new Object[3];
-    int index = 0;
-    Integer[] dim = new Integer[this.dimensionCount];
-    Object[] measures = new Object[this.measureCount];
-    byte[] finalByteArr = null;
-    try {
-
-      // read dimension values
-
-      for (int i = 0; i < this.dimensionCount; i++) {
-        dim[index++] = stream.readInt();
-      }
-
-      if ((this.noDictionaryCount + this.complexDimensionCount) > 0) {
-        short lengthOfByteArray = stream.readShort();
-        ByteBuffer buff = ByteBuffer.allocate(lengthOfByteArray + 2);
-        buff.putShort(lengthOfByteArray);
-        byte[] byteArr = new byte[lengthOfByteArray];
-        stream.readFully(byteArr);
-
-        buff.put(byteArr);
-        finalByteArr = buff.array();
-
-      }
-
-      index = 0;
-      // read measure values
-      for (int i = 0; i < this.measureCount; i++) {
-        if (stream.readByte() == 1) {
-          if (aggType[i] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) {
-            measures[index++] = stream.readDouble();
-          } else if (aggType[i] == CarbonCommonConstants.BIG_INT_MEASURE) {
-            measures[index++] = stream.readLong();
-          } else {
-            int len = stream.readInt();
-            byte[] buff = new byte[len];
-            stream.readFully(buff);
-            measures[index++] = buff;
-          }
-        } else {
-          measures[index++] = null;
-        }
-      }
-
-      NonDictionaryUtil.prepareOutObj(holder, dim, finalByteArr, 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;
-  }
-
-  /**
    * Reads row from file
    * @return Object[]
    * @throws CarbonSortKeyAndGroupByException
    */
-  private Object[] getRowFromStreamWithOutKettle() 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];
@@ -480,60 +402,6 @@ public class SortTempFileChunkHolder implements Comparable<SortTempFileChunkHold
   }
 
   @Override public int compareTo(SortTempFileChunkHolder other) {
-    if (useKettle) {
-      return compareWithKettle(other);
-
-    } else {
-      return compareWithOutKettle(other);
-    }
-  }
-
-  // TODO Remove after kettle flow is removed.
-  private int compareWithKettle(SortTempFileChunkHolder other) {
-    int diff = 0;
-
-    int normalIndex = 0;
-    int noDictionaryindex = 0;
-
-    for (boolean isNoDictionary : isNoDictionaryDimensionColumn) {
-
-      if (isNoDictionary) {
-        byte[] byteArr1 = (byte[]) returnRow[IgnoreDictionary.BYTE_ARRAY_INDEX_IN_ROW.getIndex()];
-
-        ByteBuffer buff1 = ByteBuffer.wrap(byteArr1);
-
-        // extract a high card dims from complete byte[].
-        NonDictionaryUtil
-            .extractSingleHighCardDims(byteArr1, noDictionaryindex, noDictionaryCount, buff1);
-
-        byte[] byteArr2 =
-            (byte[]) other.returnRow[IgnoreDictionary.BYTE_ARRAY_INDEX_IN_ROW.getIndex()];
-
-        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, returnRow);
-        int dimFieldB = NonDictionaryUtil.getDimension(normalIndex, other.returnRow);
-        diff = dimFieldA - dimFieldB;
-        if (diff != 0) {
-          return diff;
-        }
-        normalIndex++;
-      }
-    }
-    return diff;
-  }
-
-  private int compareWithOutKettle(SortTempFileChunkHolder other) {
     int diff = 0;
     int index = 0;
     int noDictionaryIndex = 0;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStep.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStep.java
deleted file mode 100644
index 03e8b25..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStep.java
+++ /dev/null
@@ -1,283 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.sortdatastep;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.common.logging.impl.StandardLogService;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.processing.schema.metadata.SortObserver;
-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.util.NonDictionaryUtil;
-
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStep;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-
-public class SortKeyStep extends BaseStep {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SortKeyStep.class.getName());
-
-  /**
-   * CarbonSortKeyAndGroupByStepData
-   */
-  private SortKeyStepData data;
-
-  /**
-   * CarbonSortKeyAndGroupByStepMeta
-   */
-  private SortKeyStepMeta meta;
-
-  /**
-   * carbonSortKeys
-   */
-  private SortDataRows sortDataRows;
-
-  /**
-   * intermediateFileMerger
-   */
-  private SortIntermediateFileMerger intermediateFileMerger;
-
-  /**
-   * rowCounter
-   */
-  private long readCounter;
-
-  /**
-   * writeCounter
-   */
-  private long writeCounter;
-
-  /**
-   * logCounter
-   */
-  private int logCounter;
-
-  /**
-   * observer
-   */
-  private SortObserver observer;
-
-  /**
-   * To determine whether the column is dictionary or not.
-   */
-  private boolean[] noDictionaryColMaping;
-
-  /**
-   * CarbonSortKeyAndGroupByStep Constructor
-   *
-   * @param stepMeta
-   * @param stepDataInterface
-   * @param copyNr
-   * @param transMeta
-   * @param trans
-   */
-  public SortKeyStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr,
-      TransMeta transMeta, Trans trans) {
-    super(stepMeta, stepDataInterface, copyNr, transMeta, trans);
-  }
-
-  /**
-   * Perform the equivalent of processing one row. Typically this means
-   * reading a row from input (getRow()) and passing a row to output
-   * (putRow)).
-   *
-   * @param smi The steps metadata to work with
-   * @param sdi The steps temporary working data to work with (database
-   *            connections, result sets, caches, temporary variables, etc.)
-   * @return false if no more rows can be processed or an error occurred.
-   * @throws KettleException
-   */
-  public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException {
-    // get step meta
-    this.meta = ((SortKeyStepMeta) smi);
-    StandardLogService.setThreadName(meta.getPartitionID(), null);
-    // get step data
-    this.data = ((SortKeyStepData) sdi);
-
-    // get row
-    Object[] row = getRow();
-
-    // create sort observer
-    this.observer = new SortObserver();
-
-    // if row is null then this step can start processing the data
-    if (row == null) {
-      return processRowToNextStep();
-    }
-
-    // check if all records are null than send empty row to next step
-    else if (NonDictionaryUtil.checkAllValuesForNull(row)) {
-      // create empty row out size
-      int outSize = Integer.parseInt(meta.getOutputRowSize());
-
-      Object[] outRow = new Object[outSize];
-
-      // clone out row meta
-      this.data.setOutputRowMeta((RowMetaInterface) getInputRowMeta().clone());
-
-      // get all fields
-      this.meta.getFields(data.getOutputRowMeta(), getStepname(), null, null, this);
-
-      LOGGER.info("Record Procerssed For table: " + meta.getTabelName());
-      LOGGER.info("Record Form Previous Step was null");
-      String logMessage = "Summary: Carbon Sort Key Step: Read: " + 1 + ": Write: " + 1;
-      LOGGER.info(logMessage);
-
-      putRow(data.getOutputRowMeta(), outRow);
-      setOutputDone();
-      return false;
-    }
-
-    // if first
-    if (first) {
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordSortRowsStepTotalTime(
-          meta.getPartitionID(), System.currentTimeMillis());
-      first = false;
-
-      // clone out row meta
-      this.data.setOutputRowMeta((RowMetaInterface) getInputRowMeta().clone());
-
-      // get all fields
-      this.meta.getFields(data.getOutputRowMeta(), getStepname(), null, null, this);
-
-      this.meta.setNoDictionaryCount(
-          NonDictionaryUtil.extractNoDictionaryCount(meta.getNoDictionaryDims()));
-
-      this.noDictionaryColMaping =
-          NonDictionaryUtil.convertStringToBooleanArr(meta.getNoDictionaryDimsMapping());
-      SortParameters parameters =
-          SortParameters.createSortParameters(meta.getDatabaseName(), meta.getTabelName(),
-              meta.getDimensionCount(), meta.getComplexDimensionCount(), meta.getMeasureCount(),
-              meta.getNoDictionaryCount(), meta.getPartitionID(),
-              meta.getSegmentId() + "", meta.getTaskNo(), this.noDictionaryColMaping);
-      intermediateFileMerger = new SortIntermediateFileMerger(parameters);
-      this.sortDataRows = new SortDataRows(parameters, intermediateFileMerger);
-      try {
-        // initialize sort
-        this.sortDataRows.initialize();
-      } catch (CarbonSortKeyAndGroupByException e) {
-        throw new KettleException(e);
-      }
-
-      this.logCounter = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.DATA_LOAD_LOG_COUNTER,
-              CarbonCommonConstants.DATA_LOAD_LOG_COUNTER_DEFAULT_COUNTER));
-    }
-
-    readCounter++;
-    if (readCounter % logCounter == 0) {
-      LOGGER.info("Record Procerssed For table: " + meta.getTabelName());
-      String logMessage = "Carbon Sort Key Step: Record Read: " + readCounter;
-      LOGGER.info(logMessage);
-    }
-
-    try {
-      // add row
-      this.sortDataRows.addRow(row);
-      writeCounter++;
-    } catch (Throwable e) {
-      LOGGER.error(e);
-      throw new KettleException(e);
-    }
-
-    return true;
-  }
-
-  /**
-   * Below method will be used to process data to next step
-   *
-   * @return false is finished
-   * @throws KettleException
-   */
-  private boolean processRowToNextStep() throws KettleException {
-    if (null == this.sortDataRows) {
-      LOGGER.info("Record Processed For table: " + meta.getTabelName());
-      LOGGER.info("Number of Records was Zero");
-      String logMessage = "Summary: Carbon Sort Key Step: Read: " + 0 + ": Write: " + 0;
-      LOGGER.info(logMessage);
-      putRow(data.getOutputRowMeta(), new Object[0]);
-      setOutputDone();
-      return false;
-    }
-
-    try {
-      // start sorting
-      this.sortDataRows.startSorting();
-      this.intermediateFileMerger.finish();
-
-      // check any more rows are present
-      LOGGER.info("Record Processed For table: " + meta.getTabelName());
-      String logMessage =
-          "Summary: Carbon Sort Key Step: Read: " + readCounter + ": Write: " + writeCounter;
-      LOGGER.info(logMessage);
-      putRow(data.getOutputRowMeta(), new Object[0]);
-      setOutputDone();
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordSortRowsStepTotalTime(
-          meta.getPartitionID(), System.currentTimeMillis());
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordDictionaryValuesTotalTime(
-          meta.getPartitionID(), System.currentTimeMillis());
-      return false;
-    } catch (CarbonSortKeyAndGroupByException e) {
-      throw new KettleException(e);
-    }
-
-  }
-
-  /**
-   * Initialize and do work where other steps need to wait for...
-   *
-   * @param smi The metadata to work with
-   * @param sdi The data to initialize
-   * @return step initialize or not
-   */
-  public boolean init(StepMetaInterface smi, StepDataInterface sdi) {
-    this.meta = ((SortKeyStepMeta) smi);
-    this.data = ((SortKeyStepData) sdi);
-    return super.init(smi, sdi);
-  }
-
-  /**
-   * Dispose of this step: close files, empty logs, etc.
-   *
-   * @param smi The metadata to work with
-   * @param sdi The data to dispose of
-   */
-  public void dispose(StepMetaInterface smi, StepDataInterface sdi) {
-    this.meta = ((SortKeyStepMeta) smi);
-    this.data = ((SortKeyStepData) sdi);
-    this.sortDataRows = null;
-    super.dispose(smi, sdi);
-    this.meta = null;
-    this.data = null;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepData.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepData.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepData.java
deleted file mode 100644
index 1a9859f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepData.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.sortandgroupby.sortdatastep;
-
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.trans.step.BaseStepData;
-import org.pentaho.di.trans.step.StepDataInterface;
-
-public class SortKeyStepData extends BaseStepData implements StepDataInterface {
-
-  /**
-   * outputRowMeta
-   */
-  private RowMetaInterface outputRowMeta;
-
-  /**
-   * rowMeta
-   */
-  private RowMetaInterface rowMeta;
-
-  public RowMetaInterface getOutputRowMeta() {
-    return outputRowMeta;
-  }
-
-  public void setOutputRowMeta(RowMetaInterface outputRowMeta) {
-    this.outputRowMeta = outputRowMeta;
-  }
-
-  public RowMetaInterface getRowMeta() {
-    return rowMeta;
-  }
-
-  public void setRowMeta(RowMetaInterface rowMeta) {
-    this.rowMeta = rowMeta;
-  }
-}
\ No newline at end of file


[13/13] incubator-carbondata git commit: [CARBONDATA-821] Removed kettle related code and refactored This closes #699

Posted by ja...@apache.org.
[CARBONDATA-821] Removed kettle related code and refactored This closes #699


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

Branch: refs/heads/master
Commit: 73190895411fc57a264ce70b6803bea77cf227bd
Parents: 3d5cf45 e6b6090
Author: jackylk <ja...@huawei.com>
Authored: Thu Mar 30 10:41:20 2017 +0530
Committer: jackylk <ja...@huawei.com>
Committed: Thu Mar 30 10:41:20 2017 +0530

----------------------------------------------------------------------
 conf/carbon.properties.template                 |    2 -
 conf/dataload.properties.template               |    7 -
 core/pom.xml                                    |   15 -
 .../core/constants/CarbonCommonConstants.java   |    7 -
 .../dataholder/CarbonWriteDataHolder.java       |    5 +-
 .../executor/impl/AbstractQueryExecutor.java    |    2 +-
 .../core/scan/executor/util/QueryUtil.java      |    2 +-
 .../core/scan/wrappers/ByteArrayWrapper.java    |    9 +
 .../apache/carbondata/core/util/CarbonUtil.java |   39 -
 .../core/writer/HierarchyValueWriterForCSV.java |  318 ---
 .../store/impl/DFSFileHolderImplUnitTest.java   |    2 -
 .../store/impl/FileHolderImplUnitTest.java      |    3 +-
 .../scan/complextypes/ArrayQueryTypeTest.java   |    4 +-
 .../scan/expression/ExpressionResultTest.java   |    4 +-
 .../DriverQueryStatisticsRecorderImplTest.java  |    2 +-
 .../carbondata/core/util/CarbonUtilTest.java    |   56 +-
 .../carbondata/examples/CarbonExample.scala     |    3 -
 .../carbondata/examples/util/ExampleUtils.scala |    7 -
 .../examples/CarbonSessionExample.scala         |    1 -
 .../examples/SparkSessionExample.scala          |    1 -
 .../hadoop/ft/CarbonInputMapperTest.java        |    3 +-
 .../hadoop/test/util/StoreCreator.java          |  242 +--
 integration/spark-common-test/pom.xml           |    1 -
 .../dataload/TestLoadDataGeneral.scala          |    2 +-
 .../allqueries/DoubleDataTypeTest.scala         |    2 +-
 .../dataload/TestLoadDataWithHiveSyntax.scala   |    3 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  134 --
 .../spark/merger/RowResultMerger.java           |    2 +-
 .../spark/merger/TupleConversionAdapter.java    |   14 +-
 .../merger/exeception/SliceMergerException.java |   78 +
 .../carbondata/spark/util/LoadMetadataUtil.java |    7 +-
 .../apache/carbondata/spark/CarbonOption.scala  |    2 -
 .../spark/rdd/CarbonDataLoadRDD.scala           |  754 -------
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |    3 -
 .../apache/carbondata/spark/rdd/Compactor.scala |    2 -
 .../spark/rdd/DataManagementFunc.scala          |    4 -
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |   86 +-
 .../carbondata/spark/rdd/UpdateDataLoad.scala   |    5 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |   52 -
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |    2 +-
 .../execution/command/carbonTableSchema.scala   |    2 -
 .../spark/sql/test/TestQueryExecutor.scala      |    1 -
 .../scala/org/apache/spark/util/SparkUtil.scala |    2 +-
 integration/spark/pom.xml                       |    1 -
 .../spark/CarbonDataFrameWriter.scala           |   12 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  112 +-
 .../execution/command/carbonTableSchema.scala   |   35 +-
 .../spark/sql/test/SparkTestQueryExecutor.scala |    4 +-
 .../BadRecordLoggerSharedDictionaryTest.scala   |   12 -
 .../dataload/TestLoadDataWithSinglePass.scala   |   10 +-
 integration/spark2/pom.xml                      |    1 -
 .../spark/rdd/CarbonDataRDDFactory.scala        |  178 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |    1 -
 .../execution/command/carbonTableSchema.scala   |   34 +-
 .../sql/test/Spark2TestQueryExecutor.scala      |    1 -
 .../org/apache/spark/util/TableLoader.scala     |    6 -
 .../bucketing/TableBucketingTestCase.scala      |   10 +-
 .../vectorreader/VectorReaderTestCase.scala     |    2 +-
 pom.xml                                         |    2 -
 .../carbonplugins/.kettle/kettle.properties     |   10 -
 .../plugin.xml                                  |   28 -
 .../carbonautoagggraphgenerator/plugin.xml      |   28 -
 .../steps/carbonautoaggslicemerger/plugin.xml   |   28 -
 .../steps/carboncsvbasedseqgen/plugin.xml       |   29 -
 .../plugins/steps/carboncsvreader/plugin.xml    |   29 -
 .../steps/carboncsvreaderstrep/plugin.xml       |   29 -
 .../plugins/steps/carbondatawriter/plugin.xml   |   27 -
 .../plugins/steps/carbonfactreader/plugin.xml   |   28 -
 .../plugins/steps/carbongroupby/plugin.xml      |   27 -
 .../steps/carboninmemoryfactreader/plugin.xml   |   27 -
 .../plugins/steps/carbonseqgen/plugin.xml       |   28 -
 .../plugins/steps/carbonslicemerger/plugin.xml  |   28 -
 .../steps/carbonsortkeyandgroupby/plugin.xml    |   28 -
 .../plugins/steps/mdkeygenstep/plugin.xml       |   28 -
 .../plugins/steps/sortkeystep/plugin.xml        |   27 -
 processing/pom.xml                              |   26 -
 .../processing/csvload/BlockDetails.java        |   84 +
 .../processing/csvload/DataGraphExecuter.java   |  475 -----
 .../processing/csvload/GraphExecutionUtil.java  |  242 ---
 .../processing/csvreaderstep/BlockDetails.java  |   85 -
 .../csvreaderstep/BoundedDataStream.java        |  124 --
 .../processing/csvreaderstep/CsvInput.java      |  431 ----
 .../processing/csvreaderstep/CsvInputData.java  |   47 -
 .../processing/csvreaderstep/CsvInputMeta.java  |  971 ---------
 .../processing/csvreaderstep/RddInputUtils.java |   42 -
 .../csvreaderstep/RddInpututilsForUpdate.java   |   41 -
 .../csvreaderstep/UnivocityCsvParser.java       |  224 --
 .../csvreaderstep/UnivocityCsvParserVo.java     |  220 --
 .../csvreaderstep/step-attributes.xml           |  229 --
 .../dataprocessor/DataProcessTaskStatus.java    |  301 ---
 .../dataprocessor/IDataProcessStatus.java       |  194 --
 .../queue/impl/RecordComparator.java            |   43 -
 .../holder/DataProcessorRecordHolder.java       |   38 -
 .../processing/datatypes/ArrayDataType.java     |   29 -
 .../processing/datatypes/GenericDataType.java   |   18 -
 .../processing/datatypes/PrimitiveDataType.java |   25 -
 .../processing/datatypes/StructDataType.java    |   38 -
 .../exception/CarbonDataProcessorException.java |   78 -
 .../graphgenerator/GraphGenerator.java          |  965 ---------
 .../graphgenerator/GraphGeneratorConstants.java |   74 -
 .../graphgenerator/GraphGeneratorException.java |   79 -
 .../configuration/GraphConfigurationInfo.java   |  894 --------
 .../processing/mdkeygen/MDKeyGenStep.java       |  531 -----
 .../processing/mdkeygen/MDKeyGenStepData.java   |   37 -
 .../processing/mdkeygen/MDKeyGenStepMeta.java   |  529 -----
 .../processing/mdkeygen/file/FileData.java      |   72 -
 .../processing/mdkeygen/file/FileManager.java   |   59 -
 .../mdkeygen/file/IFileManagerComposite.java    |   57 -
 .../mdkeygen/messages/messages_en_US.properties |   22 -
 .../merger/exeception/SliceMergerException.java |   78 -
 .../merger/step/CarbonSliceMergerStep.java      |  159 --
 .../merger/step/CarbonSliceMergerStepData.java  |   41 -
 .../merger/step/CarbonSliceMergerStepMeta.java  |  470 ----
 .../sort/impl/ParallelReadMergeSorterImpl.java  |    2 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |    3 +-
 .../steps/DataWriterProcessorStepImpl.java      |    2 +-
 .../schema/metadata/ArrayWrapper.java           |   62 -
 .../schema/metadata/ColumnSchemaDetails.java    |  100 -
 .../metadata/ColumnSchemaDetailsWrapper.java    |  108 -
 .../processing/schema/metadata/ColumnsInfo.java |  430 ----
 .../schema/metadata/HierarchiesInfo.java        |   51 -
 .../processing/schema/metadata/TableOption.java |   80 -
 .../schema/metadata/TableOptionWrapper.java     |  104 -
 .../sortdata/IntermediateFileMerger.java        |   85 +-
 .../sortandgroupby/sortdata/SortDataRows.java   |  104 +-
 .../sortandgroupby/sortdata/SortParameters.java |   15 -
 .../sortdata/SortTempFileChunkHolder.java       |  136 +-
 .../sortdatastep/SortKeyStep.java               |  283 ---
 .../sortdatastep/SortKeyStepData.java           |   51 -
 .../sortdatastep/SortKeyStepMeta.java           |  490 -----
 .../store/CarbonFactDataHandlerColumnar.java    |  296 +--
 .../store/CarbonFactDataHandlerModel.java       |   14 -
 .../store/SingleThreadFinalSortFilesMerger.java |    8 +-
 .../processing/store/file/FileData.java         |   52 +
 .../processing/store/file/FileManager.java      |   59 +
 .../store/file/IFileManagerComposite.java       |   57 +
 .../store/writer/AbstractFactDataWriter.java    |    2 +-
 .../store/writer/CarbonDataWriterVo.java        |    2 +-
 .../CarbonCSVBasedDimSurrogateKeyGen.java       |  495 -----
 .../csvbased/CarbonCSVBasedSeqGenData.java      |  136 --
 .../csvbased/CarbonCSVBasedSeqGenMeta.java      | 1436 -------------
 .../csvbased/CarbonCSVBasedSeqGenStep.java      | 2009 ------------------
 .../FileStoreSurrogateKeyGenForCSV.java         |  339 ---
 .../dbbased/messages/messages_en_US.properties  |   61 -
 .../util/CarbonDataProcessorUtil.java           |   53 -
 .../processing/util/CarbonSchemaParser.java     | 1076 ----------
 .../hadoopfileinput/HadoopFileInputMeta.java    |   27 -
 .../messages/messages_en_US.properties          |   18 -
 .../carbon/datastore/BlockIndexStoreTest.java   |    2 +-
 .../carbondata/lcm/locks/LocalFileLockTest.java |    2 +-
 .../carbondata/processing/StoreCreator.java     |  506 +++++
 .../carbondata/test/util/StoreCreator.java      |  539 -----
 152 files changed, 1164 insertions(+), 18945 deletions(-)
----------------------------------------------------------------------



[12/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
Removed kettle related code and refactored

Removed carbonplugins

Added back method

Fixed test

Fixed test


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

Branch: refs/heads/master
Commit: e6b60907f0be2efd89884b81490a112ef71fd9cd
Parents: 3d5cf45
Author: ravipesala <ra...@gmail.com>
Authored: Sun Mar 26 16:10:47 2017 +0530
Committer: jackylk <ja...@huawei.com>
Committed: Thu Mar 30 10:34:14 2017 +0530

----------------------------------------------------------------------
 conf/carbon.properties.template                 |    2 -
 conf/dataload.properties.template               |    7 -
 core/pom.xml                                    |   15 -
 .../core/constants/CarbonCommonConstants.java   |    7 -
 .../dataholder/CarbonWriteDataHolder.java       |    5 +-
 .../executor/impl/AbstractQueryExecutor.java    |    2 +-
 .../core/scan/executor/util/QueryUtil.java      |    2 +-
 .../core/scan/wrappers/ByteArrayWrapper.java    |    9 +
 .../apache/carbondata/core/util/CarbonUtil.java |   39 -
 .../core/writer/HierarchyValueWriterForCSV.java |  318 ---
 .../store/impl/DFSFileHolderImplUnitTest.java   |    2 -
 .../store/impl/FileHolderImplUnitTest.java      |    3 +-
 .../scan/complextypes/ArrayQueryTypeTest.java   |    4 +-
 .../scan/expression/ExpressionResultTest.java   |    4 +-
 .../DriverQueryStatisticsRecorderImplTest.java  |    2 +-
 .../carbondata/core/util/CarbonUtilTest.java    |   56 +-
 .../carbondata/examples/CarbonExample.scala     |    3 -
 .../carbondata/examples/util/ExampleUtils.scala |    7 -
 .../examples/CarbonSessionExample.scala         |    1 -
 .../examples/SparkSessionExample.scala          |    1 -
 .../hadoop/ft/CarbonInputMapperTest.java        |    3 +-
 .../hadoop/test/util/StoreCreator.java          |  242 +--
 integration/spark-common-test/pom.xml           |    1 -
 .../dataload/TestLoadDataGeneral.scala          |    2 +-
 .../allqueries/DoubleDataTypeTest.scala         |    2 +-
 .../dataload/TestLoadDataWithHiveSyntax.scala   |    3 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  134 --
 .../spark/merger/RowResultMerger.java           |    2 +-
 .../spark/merger/TupleConversionAdapter.java    |   14 +-
 .../merger/exeception/SliceMergerException.java |   78 +
 .../carbondata/spark/util/LoadMetadataUtil.java |    7 +-
 .../apache/carbondata/spark/CarbonOption.scala  |    2 -
 .../spark/rdd/CarbonDataLoadRDD.scala           |  754 -------
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |    3 -
 .../apache/carbondata/spark/rdd/Compactor.scala |    2 -
 .../spark/rdd/DataManagementFunc.scala          |    4 -
 .../spark/rdd/NewCarbonDataLoadRDD.scala        |   86 +-
 .../carbondata/spark/rdd/UpdateDataLoad.scala   |    5 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |   52 -
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala |    2 +-
 .../execution/command/carbonTableSchema.scala   |    2 -
 .../spark/sql/test/TestQueryExecutor.scala      |    1 -
 .../scala/org/apache/spark/util/SparkUtil.scala |    2 +-
 integration/spark/pom.xml                       |    1 -
 .../spark/CarbonDataFrameWriter.scala           |   12 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  112 +-
 .../execution/command/carbonTableSchema.scala   |   35 +-
 .../spark/sql/test/SparkTestQueryExecutor.scala |    4 +-
 .../BadRecordLoggerSharedDictionaryTest.scala   |   12 -
 .../dataload/TestLoadDataWithSinglePass.scala   |   10 +-
 integration/spark2/pom.xml                      |    1 -
 .../spark/rdd/CarbonDataRDDFactory.scala        |  178 +-
 .../spark/sql/CarbonDataFrameWriter.scala       |    1 -
 .../execution/command/carbonTableSchema.scala   |   34 +-
 .../sql/test/Spark2TestQueryExecutor.scala      |    1 -
 .../org/apache/spark/util/TableLoader.scala     |    6 -
 .../bucketing/TableBucketingTestCase.scala      |   10 +-
 .../vectorreader/VectorReaderTestCase.scala     |    2 +-
 pom.xml                                         |    2 -
 .../carbonplugins/.kettle/kettle.properties     |   10 -
 .../plugin.xml                                  |   28 -
 .../carbonautoagggraphgenerator/plugin.xml      |   28 -
 .../steps/carbonautoaggslicemerger/plugin.xml   |   28 -
 .../steps/carboncsvbasedseqgen/plugin.xml       |   29 -
 .../plugins/steps/carboncsvreader/plugin.xml    |   29 -
 .../steps/carboncsvreaderstrep/plugin.xml       |   29 -
 .../plugins/steps/carbondatawriter/plugin.xml   |   27 -
 .../plugins/steps/carbonfactreader/plugin.xml   |   28 -
 .../plugins/steps/carbongroupby/plugin.xml      |   27 -
 .../steps/carboninmemoryfactreader/plugin.xml   |   27 -
 .../plugins/steps/carbonseqgen/plugin.xml       |   28 -
 .../plugins/steps/carbonslicemerger/plugin.xml  |   28 -
 .../steps/carbonsortkeyandgroupby/plugin.xml    |   28 -
 .../plugins/steps/mdkeygenstep/plugin.xml       |   28 -
 .../plugins/steps/sortkeystep/plugin.xml        |   27 -
 processing/pom.xml                              |   26 -
 .../processing/csvload/BlockDetails.java        |   84 +
 .../processing/csvload/DataGraphExecuter.java   |  475 -----
 .../processing/csvload/GraphExecutionUtil.java  |  242 ---
 .../processing/csvreaderstep/BlockDetails.java  |   85 -
 .../csvreaderstep/BoundedDataStream.java        |  124 --
 .../processing/csvreaderstep/CsvInput.java      |  431 ----
 .../processing/csvreaderstep/CsvInputData.java  |   47 -
 .../processing/csvreaderstep/CsvInputMeta.java  |  971 ---------
 .../processing/csvreaderstep/RddInputUtils.java |   42 -
 .../csvreaderstep/RddInpututilsForUpdate.java   |   41 -
 .../csvreaderstep/UnivocityCsvParser.java       |  224 --
 .../csvreaderstep/UnivocityCsvParserVo.java     |  220 --
 .../csvreaderstep/step-attributes.xml           |  229 --
 .../dataprocessor/DataProcessTaskStatus.java    |  301 ---
 .../dataprocessor/IDataProcessStatus.java       |  194 --
 .../queue/impl/RecordComparator.java            |   43 -
 .../holder/DataProcessorRecordHolder.java       |   38 -
 .../processing/datatypes/ArrayDataType.java     |   29 -
 .../processing/datatypes/GenericDataType.java   |   18 -
 .../processing/datatypes/PrimitiveDataType.java |   25 -
 .../processing/datatypes/StructDataType.java    |   38 -
 .../exception/CarbonDataProcessorException.java |   78 -
 .../graphgenerator/GraphGenerator.java          |  965 ---------
 .../graphgenerator/GraphGeneratorConstants.java |   74 -
 .../graphgenerator/GraphGeneratorException.java |   79 -
 .../configuration/GraphConfigurationInfo.java   |  894 --------
 .../processing/mdkeygen/MDKeyGenStep.java       |  531 -----
 .../processing/mdkeygen/MDKeyGenStepData.java   |   37 -
 .../processing/mdkeygen/MDKeyGenStepMeta.java   |  529 -----
 .../processing/mdkeygen/file/FileData.java      |   72 -
 .../processing/mdkeygen/file/FileManager.java   |   59 -
 .../mdkeygen/file/IFileManagerComposite.java    |   57 -
 .../mdkeygen/messages/messages_en_US.properties |   22 -
 .../merger/exeception/SliceMergerException.java |   78 -
 .../merger/step/CarbonSliceMergerStep.java      |  159 --
 .../merger/step/CarbonSliceMergerStepData.java  |   41 -
 .../merger/step/CarbonSliceMergerStepMeta.java  |  470 ----
 .../sort/impl/ParallelReadMergeSorterImpl.java  |    2 +-
 ...arallelReadMergeSorterWithBucketingImpl.java |    3 +-
 .../steps/DataWriterProcessorStepImpl.java      |    2 +-
 .../schema/metadata/ArrayWrapper.java           |   62 -
 .../schema/metadata/ColumnSchemaDetails.java    |  100 -
 .../metadata/ColumnSchemaDetailsWrapper.java    |  108 -
 .../processing/schema/metadata/ColumnsInfo.java |  430 ----
 .../schema/metadata/HierarchiesInfo.java        |   51 -
 .../processing/schema/metadata/TableOption.java |   80 -
 .../schema/metadata/TableOptionWrapper.java     |  104 -
 .../sortdata/IntermediateFileMerger.java        |   85 +-
 .../sortandgroupby/sortdata/SortDataRows.java   |  104 +-
 .../sortandgroupby/sortdata/SortParameters.java |   15 -
 .../sortdata/SortTempFileChunkHolder.java       |  136 +-
 .../sortdatastep/SortKeyStep.java               |  283 ---
 .../sortdatastep/SortKeyStepData.java           |   51 -
 .../sortdatastep/SortKeyStepMeta.java           |  490 -----
 .../store/CarbonFactDataHandlerColumnar.java    |  296 +--
 .../store/CarbonFactDataHandlerModel.java       |   14 -
 .../store/SingleThreadFinalSortFilesMerger.java |    8 +-
 .../processing/store/file/FileData.java         |   52 +
 .../processing/store/file/FileManager.java      |   59 +
 .../store/file/IFileManagerComposite.java       |   57 +
 .../store/writer/AbstractFactDataWriter.java    |    2 +-
 .../store/writer/CarbonDataWriterVo.java        |    2 +-
 .../CarbonCSVBasedDimSurrogateKeyGen.java       |  495 -----
 .../csvbased/CarbonCSVBasedSeqGenData.java      |  136 --
 .../csvbased/CarbonCSVBasedSeqGenMeta.java      | 1436 -------------
 .../csvbased/CarbonCSVBasedSeqGenStep.java      | 2009 ------------------
 .../FileStoreSurrogateKeyGenForCSV.java         |  339 ---
 .../dbbased/messages/messages_en_US.properties  |   61 -
 .../util/CarbonDataProcessorUtil.java           |   53 -
 .../processing/util/CarbonSchemaParser.java     | 1076 ----------
 .../hadoopfileinput/HadoopFileInputMeta.java    |   27 -
 .../messages/messages_en_US.properties          |   18 -
 .../carbon/datastore/BlockIndexStoreTest.java   |    2 +-
 .../carbondata/lcm/locks/LocalFileLockTest.java |    2 +-
 .../carbondata/processing/StoreCreator.java     |  506 +++++
 .../carbondata/test/util/StoreCreator.java      |  539 -----
 152 files changed, 1164 insertions(+), 18945 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/conf/carbon.properties.template
----------------------------------------------------------------------
diff --git a/conf/carbon.properties.template b/conf/carbon.properties.template
index ac2d20e..b5f5101 100644
--- a/conf/carbon.properties.template
+++ b/conf/carbon.properties.template
@@ -23,8 +23,6 @@ carbon.storelocation=hdfs://hacluster/Opt/CarbonStore
 carbon.ddl.base.hdfs.url=hdfs://hacluster/opt/data
 #Path where the bad records are stored
 carbon.badRecords.location=/opt/Carbon/Spark/badrecords
-#Mandatory. path to kettle home
-carbon.kettle.home=$<SPARK_HOME>/carbonlib/carbonplugins
 
 #################### Performance Configuration ##################
 ######## DataLoading Configuration ########

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/conf/dataload.properties.template
----------------------------------------------------------------------
diff --git a/conf/dataload.properties.template b/conf/dataload.properties.template
index cfafb4c..cab712a 100644
--- a/conf/dataload.properties.template
+++ b/conf/dataload.properties.template
@@ -20,13 +20,6 @@
 # you should change to the code path of your local machine
 carbon.storelocation=/home/david/Documents/incubator-carbondata/examples/spark2/target/store
 
-#true: use kettle to load data
-#false: use new flow to load data
-use_kettle=true
-
-# you should change to the code path of your local machine
-carbon.kettle.home=/home/david/Documents/incubator-carbondata/processing/carbonplugins
-
 #csv delimiter character
 delimiter=,
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 5e46af3..976f8d2 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -45,21 +45,6 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>pentaho-kettle</groupId>
-      <artifactId>kettle-engine</artifactId>
-      <version>${kettle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>pentaho-kettle</groupId>
-      <artifactId>kettle-core</artifactId>
-      <version>${kettle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>pentaho-kettle</groupId>
-      <artifactId>kettle-db</artifactId>
-      <version>${kettle.version}</version>
-    </dependency>
-    <dependency>
       <groupId>com.google.code.gson</groupId>
       <artifactId>gson</artifactId>
       <version>2.3.1</version>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index d13bc79..789c321 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -69,10 +69,6 @@ public final class CarbonCommonConstants {
    */
   public static final String STORE_LOCATION_DEFAULT_VAL = "../carbon.store";
   /**
-   * the folder name of kettle home path
-   */
-  public static final String KETTLE_HOME_NAME = "carbonplugins";
-  /**
    * CARDINALITY_INCREMENT_DEFAULT_VALUE
    */
   public static final int CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL = 10;
@@ -1174,9 +1170,6 @@ public final class CarbonCommonConstants {
 
   public static final String LOCAL_FILE_PREFIX = "file://";
 
-  public static final String USE_KETTLE = "use_kettle";
-
-  public static final String USE_KETTLE_DEFAULT = "false";
   public static final String CARBON_CUSTOM_BLOCK_DISTRIBUTION = "carbon.custom.block.distribution";
   public static final String CARBON_CUSTOM_BLOCK_DISTRIBUTION_DEFAULT = "false";
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java b/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java
index e1aa0a0..fb21d95 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/dataholder/CarbonWriteDataHolder.java
@@ -34,7 +34,7 @@ public class CarbonWriteDataHolder {
   private byte[][] byteValues;
 
   /**
-   * byteValues for no dictionary and non kettle flow.
+   * byteValues for no dictionary.
    */
   private byte[][][] byteValuesForNonDictionary;
 
@@ -72,7 +72,6 @@ public class CarbonWriteDataHolder {
 
   /**
    * Method to initialise double array
-   * TODO Remove after kettle flow got removed.
    *
    * @param size
    */
@@ -90,7 +89,7 @@ public class CarbonWriteDataHolder {
    *
    * @param size
    */
-  public void initialiseByteArrayValuesWithOutKettle(int size) {
+  public void initialiseByteArrayValuesForKey(int size) {
     if (size < 1) {
       throw new IllegalArgumentException("Invalid array size");
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
index 2a5c342..1ddb3e6 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/impl/AbstractQueryExecutor.java
@@ -345,7 +345,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     blockExecutionInfo
         .setFixedLengthKeySize(getKeySize(currentBlockQueryDimensions, segmentProperties));
     Set<Integer> dictionaryColumnBlockIndex = new HashSet<Integer>();
-    List<Integer> noDictionaryColumnBlockIndex = new ArrayList<Integer>();
+    Set<Integer> noDictionaryColumnBlockIndex = new HashSet<Integer>();
     // get the block index to be read from file for query dimension
     // for both dictionary columns and no dictionary columns
     QueryUtil.fillQueryDimensionsBlockIndexes(currentBlockQueryDimensions,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
index 5a98e44..be5e8a4 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/executor/util/QueryUtil.java
@@ -607,7 +607,7 @@ public class QueryUtil {
    */
   public static void fillQueryDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
       Map<Integer, Integer> columnOrdinalToBlockIndexMapping,
-      Set<Integer> dictionaryDimensionBlockIndex, List<Integer> noDictionaryDimensionBlockIndex) {
+      Set<Integer> dictionaryDimensionBlockIndex, Set<Integer> noDictionaryDimensionBlockIndex) {
     for (QueryDimension queryDimension : queryDimensions) {
       if (CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)
           && queryDimension.getDimension().numberOfChild() == 0) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
index ff82fc6..93bf8eb 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/wrappers/ByteArrayWrapper.java
@@ -82,6 +82,15 @@ public class ByteArrayWrapper implements Comparable<ByteArrayWrapper> {
   /**
    * to get the no dictionary column data
    *
+   * @return no dictionary keys
+   */
+  public byte[][] getNoDictionaryKeys() {
+    return this.noDictionaryKeys;
+  }
+
+  /**
+   * to get the no dictionary column data
+   *
    * @param index of the no dictionary key
    * @return no dictionary key for the index
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 6d510f6..bf8c03b 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -23,12 +23,10 @@ import java.io.ByteArrayOutputStream;
 import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.ObjectInputStream;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
 import java.nio.charset.Charset;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
@@ -83,7 +81,6 @@ import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TCompactProtocol;
 import org.apache.thrift.protocol.TProtocol;
 import org.apache.thrift.transport.TIOStreamTransport;
-import org.pentaho.di.core.exception.KettleException;
 
 public final class CarbonUtil {
 
@@ -631,42 +628,6 @@ public final class CarbonUtil {
     return cardinality;
   }
 
-  public static void writeLevelCardinalityFile(String loadFolderLoc, String tableName,
-      int[] dimCardinality) throws KettleException {
-    String levelCardinalityFilePath =
-        loadFolderLoc + File.separator + CarbonCommonConstants.LEVEL_METADATA_FILE + tableName
-            + CarbonCommonConstants.CARBON_METADATA_EXTENSION;
-    FileOutputStream fileOutputStream = null;
-    FileChannel channel = null;
-    try {
-      int dimCardinalityArrLength = dimCardinality.length;
-
-      // first four bytes for writing the length of array, remaining for array data
-      ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE
-          + dimCardinalityArrLength * CarbonCommonConstants.INT_SIZE_IN_BYTE);
-
-      fileOutputStream = new FileOutputStream(levelCardinalityFilePath);
-      channel = fileOutputStream.getChannel();
-      buffer.putInt(dimCardinalityArrLength);
-
-      for (int i = 0; i < dimCardinalityArrLength; i++) {
-        buffer.putInt(dimCardinality[i]);
-      }
-
-      buffer.flip();
-      channel.write(buffer);
-      buffer.clear();
-
-      LOGGER.info("Level cardinality file written to : " + levelCardinalityFilePath);
-    } catch (IOException e) {
-      LOGGER.error("Error while writing level cardinality file : " + levelCardinalityFilePath + e
-          .getMessage());
-      throw new KettleException("Not able to write level cardinality file", e);
-    } finally {
-      closeStreams(channel, fileOutputStream);
-    }
-  }
-
   /**
    * From beeline if a delimeter is passed as \001, in code we get it as
    * escaped string as \\001. So this method will unescape the slash again and

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/main/java/org/apache/carbondata/core/writer/HierarchyValueWriterForCSV.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/HierarchyValueWriterForCSV.java b/core/src/main/java/org/apache/carbondata/core/writer/HierarchyValueWriterForCSV.java
deleted file mode 100644
index 1e427aa..0000000
--- a/core/src/main/java/org/apache/carbondata/core/writer/HierarchyValueWriterForCSV.java
+++ /dev/null
@@ -1,318 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.writer;
-
-import java.io.Closeable;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-import org.pentaho.di.core.exception.KettleException;
-
-public class HierarchyValueWriterForCSV {
-
-  /**
-   * Comment for <code>LOGGER</code>
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(HierarchyValueWriterForCSV.class.getName());
-  /**
-   * hierarchyName
-   */
-  private String hierarchyName;
-
-  /**
-   * bufferedOutStream
-   */
-  private FileChannel outPutFileChannel;
-
-  /**
-   * storeFolderLocation
-   */
-  private String storeFolderLocation;
-
-  /**
-   * intialized
-   */
-  private boolean intialized;
-
-  /**
-   * counter the number of files.
-   */
-  private int counter;
-
-  /**
-   * byteArrayList
-   */
-  private List<ByteArrayHolder> byteArrayholder =
-      new ArrayList<ByteArrayHolder>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  /**
-   * toflush
-   */
-  private int toflush;
-
-  public HierarchyValueWriterForCSV(String hierarchy, String storeFolderLocation) {
-    this.hierarchyName = hierarchy;
-    this.storeFolderLocation = storeFolderLocation;
-
-    CarbonProperties instance = CarbonProperties.getInstance();
-
-    this.toflush = Integer.parseInt(instance
-        .getProperty(CarbonCommonConstants.SORT_SIZE, CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL));
-
-    int rowSetSize = Integer.parseInt(instance.getProperty(CarbonCommonConstants.GRAPH_ROWSET_SIZE,
-        CarbonCommonConstants.GRAPH_ROWSET_SIZE_DEFAULT));
-
-    if (this.toflush > rowSetSize) {
-      this.toflush = rowSetSize;
-    }
-
-    updateCounter(hierarchy, storeFolderLocation);
-  }
-
-  /**
-   * @return Returns the byteArrayList.
-   */
-  public List<ByteArrayHolder> getByteArrayList() {
-    return byteArrayholder;
-  }
-
-  public FileChannel getBufferedOutStream() {
-    return outPutFileChannel;
-  }
-
-  private void updateCounter(final String meString, String storeFolderLocation) {
-    File storeFolder = new File(storeFolderLocation);
-
-    File[] listFiles = storeFolder.listFiles(new FileFilter() {
-
-      @Override public boolean accept(File file) {
-        if (file.getName().indexOf(meString) > -1)
-
-        {
-          return true;
-        }
-        return false;
-      }
-    });
-
-    if (null == listFiles || listFiles.length == 0) {
-      counter = 0;
-      return;
-    }
-
-    for (File hierFile : listFiles) {
-      String hierFileName = hierFile.getName();
-
-      if (hierFileName.endsWith(CarbonCommonConstants.FILE_INPROGRESS_STATUS)) {
-        hierFileName = hierFileName.substring(0, hierFileName.lastIndexOf('.'));
-        try {
-          counter = Integer.parseInt(hierFileName.substring(hierFileName.length() - 1));
-        } catch (NumberFormatException nfe) {
-
-          if (new File(hierFileName + '0' + CarbonCommonConstants.LEVEL_FILE_EXTENSION).exists()) {
-            // Need to skip because the case can come in which server went down while files were
-            // merging and the other hierarchy files were not deleted, and the current file
-            // status is inrogress. so again we will merge the files and rename to normal file
-            LOGGER.info("Need to skip as this can be case in which hierarchy file already renamed");
-            if (hierFile.delete()) {
-              LOGGER.info("Deleted the Inprogress hierarchy Files.");
-            }
-          } else {
-            // levelfileName0.level file not exist that means files is merged and other
-            // files got deleted. while renaming this file from inprogress to normal file,
-            // server got restarted/killed. so we need to rename the file to normal.
-
-            File inprogressFile = new File(storeFolder + File.separator + hierFile.getName());
-            File changetoName = new File(storeFolder + File.separator + hierFileName);
-
-            if (inprogressFile.renameTo(changetoName)) {
-              LOGGER.info(
-                  "Renaming the level Files while creating the new instance on server startup.");
-            }
-
-          }
-
-        }
-      }
-
-      String val = hierFileName.substring(hierFileName.length() - 1);
-
-      int parsedVal = getIntValue(val);
-
-      if (counter < parsedVal) {
-        counter = parsedVal;
-      }
-    }
-    counter++;
-  }
-
-  private int getIntValue(String val) {
-    int parsedVal = 0;
-    try {
-      parsedVal = Integer.parseInt(val);
-    } catch (NumberFormatException nfe) {
-      LOGGER.info("Hierarchy File is already renamed so there will not be"
-              + "any need to keep the counter");
-    }
-    return parsedVal;
-  }
-
-  private void intialize() throws KettleException {
-    intialized = true;
-
-    File f = new File(storeFolderLocation + File.separator + hierarchyName + counter
-        + CarbonCommonConstants.FILE_INPROGRESS_STATUS);
-
-    counter++;
-
-    FileOutputStream fos = null;
-
-    boolean isFileCreated = false;
-    if (!f.exists()) {
-      try {
-        isFileCreated = f.createNewFile();
-
-      } catch (IOException e) {
-        //not required: findbugs fix
-        throw new KettleException("unable to create member mapping file", e);
-      }
-      if (!isFileCreated) {
-        throw new KettleException("unable to create file" + f.getAbsolutePath());
-      }
-    }
-
-    try {
-      fos = new FileOutputStream(f);
-
-      outPutFileChannel = fos.getChannel();
-    } catch (FileNotFoundException e) {
-      closeStreamAndDeleteFile(f, outPutFileChannel, fos);
-      throw new KettleException("member Mapping File not found to write mapping info", e);
-    }
-  }
-
-  public void writeIntoHierarchyFile(byte[] bytes, int primaryKey) throws KettleException {
-    if (!intialized) {
-      intialize();
-    }
-
-    ByteBuffer byteBuffer = storeValueInCache(bytes, primaryKey);
-
-    try {
-      byteBuffer.flip();
-      outPutFileChannel.write(byteBuffer);
-    } catch (IOException e) {
-      throw new KettleException("Error while writting in the hierarchy mapping file", e);
-    }
-  }
-
-  private ByteBuffer storeValueInCache(byte[] bytes, int primaryKey) {
-
-    // adding 4 to store the total length of the row at the beginning
-    ByteBuffer buffer = ByteBuffer.allocate(bytes.length + 4);
-
-    buffer.put(bytes);
-    buffer.putInt(primaryKey);
-
-    return buffer;
-  }
-
-  public void performRequiredOperation() throws KettleException {
-    if (byteArrayholder.size() == 0) {
-      return;
-    }
-    //write to the file and close the stream.
-    Collections.sort(byteArrayholder);
-
-    for (ByteArrayHolder byteArray : byteArrayholder) {
-      writeIntoHierarchyFile(byteArray.getMdKey(), byteArray.getPrimaryKey());
-    }
-
-    CarbonUtil.closeStreams(outPutFileChannel);
-
-    //rename the inprogress file to normal .level file
-    String filePath = this.storeFolderLocation + File.separator + hierarchyName + (counter - 1)
-        + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-    File inProgressFile = new File(filePath);
-    String inprogressFileName = inProgressFile.getName();
-
-    String changedFileName = inprogressFileName.substring(0, inprogressFileName.lastIndexOf('.'));
-
-    File orgFinalName = new File(this.storeFolderLocation + File.separator + changedFileName);
-
-    if (!inProgressFile.renameTo(orgFinalName)) {
-      LOGGER.error("Not able to rename file : " + inprogressFileName);
-    }
-
-    //create the new outputStream
-    try {
-      intialize();
-    } catch (KettleException e) {
-      LOGGER.error("Not able to create output stream for file:" + hierarchyName + (counter - 1));
-    }
-
-    //clear the byte array holder also.
-    byteArrayholder.clear();
-  }
-
-  private void closeStreamAndDeleteFile(File f, Closeable... streams) {
-    boolean isDeleted = false;
-    for (Closeable stream : streams) {
-      if (null != stream) {
-        try {
-          stream.close();
-        } catch (IOException e) {
-          LOGGER.error(e, "unable to close the stream ");
-        }
-
-      }
-    }
-
-    // delete the file
-    isDeleted = f.delete();
-    if (!isDeleted) {
-      LOGGER.error("Unable to delete the file " + f.getAbsolutePath());
-    }
-
-  }
-
-  public String getHierarchyName() {
-    return hierarchyName;
-  }
-
-  public int getCounter() {
-    return counter;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
index fed712e..5c51c87 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/DFSFileHolderImplUnitTest.java
@@ -35,9 +35,7 @@ import org.junit.Test;
 
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
-import static org.pentaho.di.core.util.Assert.assertNull;
 
 public class DFSFileHolderImplUnitTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
index da722cf..ed50d63 100644
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/carbon/datastorage/filesystem/store/impl/FileHolderImplUnitTest.java
@@ -35,9 +35,8 @@ import org.junit.Test;
 
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
-import static org.pentaho.di.core.util.Assert.assertNull;
 
 public class FileHolderImplUnitTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java
index 9f37f78..e882f4e 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/complextypes/ArrayQueryTypeTest.java
@@ -21,8 +21,8 @@ import java.nio.ByteBuffer;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import static org.pentaho.di.core.util.Assert.assertNotNull;
-import static org.pentaho.di.core.util.Assert.assertNull;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 
 public class ArrayQueryTypeTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java b/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
index 30c8236..2cdb46e 100644
--- a/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/scan/expression/ExpressionResultTest.java
@@ -35,8 +35,8 @@ import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.Is.is;
 import static org.hamcrest.core.IsEqual.equalTo;
 import static org.junit.Assert.assertEquals;
-import static org.pentaho.di.core.util.Assert.assertFalse;
-import static org.pentaho.di.core.util.Assert.assertNull;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 
 public class ExpressionResultTest {
   private static final double DELTA = 1e-15;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java b/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java
index 90cae83..bb892a9 100644
--- a/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/stats/DriverQueryStatisticsRecorderImplTest.java
@@ -23,7 +23,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static junit.framework.TestCase.assertNotNull;
-import static org.pentaho.di.core.util.Assert.assertNull;
+import static org.junit.Assert.assertNull;
 
 public class DriverQueryStatisticsRecorderImplTest {
   private static DriverQueryStatisticsRecorderImpl driverQueryStatisticsRecorderImpl = null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
index 9beaac7..9adf4d4 100644
--- a/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/util/CarbonUtilTest.java
@@ -16,12 +16,26 @@
  */
 package org.apache.carbondata.core.util;
 
-import mockit.Mock;
-import mockit.MockUp;
+import java.io.BufferedReader;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 
-import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
 import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
+import org.apache.carbondata.core.datastore.compression.WriterCompressModel;
+import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.metadata.blocklet.datachunk.DataChunk;
 import org.apache.carbondata.core.metadata.datatype.DataType;
@@ -29,29 +43,20 @@ import org.apache.carbondata.core.metadata.encoder.Encoding;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.datastore.columnar.ColumnGroupModel;
-import org.apache.carbondata.core.datastore.compression.WriterCompressModel;
-import org.apache.carbondata.core.datastore.filesystem.LocalCarbonFile;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.core.metadata.ValueEncoderMeta;
 import org.apache.carbondata.core.scan.model.QueryDimension;
 
+import mockit.Mock;
+import mockit.MockUp;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.pentaho.di.core.exception.KettleException;
-
-import java.io.*;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
 
-import static org.hamcrest.MatcherAssert.assertThat;
-import static junit.framework.TestCase.*;
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertTrue;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
 
 public class CarbonUtilTest {
 
@@ -300,23 +305,6 @@ public class CarbonUtilTest {
     assertEquals(result, 5);
   }
 
-  @Test public void testToWriteLevelCardinalityFile() throws KettleException {
-    int[] dimCardinality = { 10, 20, 30, 40 };
-    CarbonUtil.writeLevelCardinalityFile("../core/src/test/resources/testDatabase", "testTable",
-        dimCardinality);
-    assertTrue(new File("../core/src/test/resources/testDatabase/levelmetadata_testTable.metadata")
-        .exists());
-  }
-
-  @Test public void testToGetCardinalityFromLevelMetadataFile()
-      throws IOException, InterruptedException {
-    int[] cardinality = CarbonUtil.getCardinalityFromLevelMetadataFile(
-        "../core/src/test/resources/testDatabase/levelmetadata_testTable.metadata");
-    int[] expectedCardinality = { 10, 20, 30, 40 };
-    for (int i = 0; i < cardinality.length; i++) {
-      assertEquals(cardinality[i], expectedCardinality[i]);
-    }
-  }
 
   @Test public void testToGetCardinalityFromLevelMetadataFileForInvalidPath()
       throws IOException, InterruptedException {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala b/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala
index b1d9e41..36013a8 100644
--- a/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala
+++ b/examples/spark/src/main/scala/org/apache/carbondata/examples/CarbonExample.scala
@@ -41,9 +41,6 @@ object CarbonExample {
            STORED BY 'carbondata'
            """)
 
-    // Currently there are two data loading flows in CarbonData, one uses Kettle as ETL tool
-    // in each node to do data loading, another uses a multi-thread framework without Kettle (See
-    // AbstractDataLoadProcessorStep)
     // Load data
     cc.sql(s"""
            LOAD DATA LOCAL INPATH '$testData' into table t3

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala
----------------------------------------------------------------------
diff --git a/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala b/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala
index 41912bc..f98ec3b 100644
--- a/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala
+++ b/examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala
@@ -31,7 +31,6 @@ object ExampleUtils {
   def currentPath: String = new File(this.getClass.getResource("/").getPath + "../../")
       .getCanonicalPath
   val storeLocation = currentPath + "/target/store"
-  val kettleHome = new File(currentPath + "/../../processing/carbonplugins").getCanonicalPath
 
   def createCarbonContext(appName: String): CarbonContext = {
     val sc = new SparkContext(new SparkConf()
@@ -44,12 +43,7 @@ object ExampleUtils {
     val cc = new CarbonContext(sc, storeLocation, currentPath + "/target/carbonmetastore")
 
     CarbonProperties.getInstance()
-      .addProperty("carbon.kettle.home", kettleHome)
       .addProperty("carbon.storelocation", storeLocation)
-    // whether use table split partition
-    // true -> use table split partition, support multiple partition loading
-    // false -> use node split partition, support data load by host partition
-    CarbonProperties.getInstance().addProperty("carbon.table.split.partition.enable", "false")
     cc
   }
 
@@ -90,7 +84,6 @@ object ExampleUtils {
       .format("carbondata")
       .option("tableName", tableName)
       .option("compress", "true")
-      .option("use_kettle", "false")
       .option("tempCSV", "false")
       .mode(mode)
       .save()

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
index 1c3b7f0..2ae9341 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala
@@ -34,7 +34,6 @@ object CarbonSessionExample {
     val metastoredb = s"$rootPath/examples/spark2/target"
 
     CarbonProperties.getInstance()
-      .addProperty("carbon.kettle.home", s"$rootPath/processing/carbonplugins")
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
 
     import org.apache.spark.sql.CarbonSession._

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala
----------------------------------------------------------------------
diff --git a/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala b/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala
index 381b2e8..63e680a 100644
--- a/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala
+++ b/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala
@@ -54,7 +54,6 @@ object SparkSessionExample {
         .getOrCreate()
 
     CarbonProperties.getInstance()
-      .addProperty("carbon.kettle.home", s"$rootPath/processing/carbonplugins")
       .addProperty("carbon.storelocation", storeLocation)
 
     spark.sparkContext.setLogLevel("WARN")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
index b55875f..0d751fc 100644
--- a/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
+++ b/hadoop/src/test/java/org/apache/carbondata/hadoop/ft/CarbonInputMapperTest.java
@@ -28,11 +28,11 @@ import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.hadoop.CarbonInputFormat;
 import org.apache.carbondata.hadoop.CarbonProjection;
-import org.apache.carbondata.hadoop.test.util.StoreCreator;
 import org.apache.carbondata.core.scan.expression.ColumnExpression;
 import org.apache.carbondata.core.scan.expression.Expression;
 import org.apache.carbondata.core.scan.expression.LiteralExpression;
 import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression;
+import org.apache.carbondata.hadoop.test.util.StoreCreator;
 
 import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
@@ -87,6 +87,7 @@ public class CarbonInputMapperTest extends TestCase {
       Assert.assertEquals("Count lines are not matching", 1000, countTheLines(outPath));
       Assert.assertEquals("Column count are not matching", 3, countTheColumns(outPath));
     } catch (Exception e) {
+      e.printStackTrace();
       Assert.assertTrue("failed", false);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 532cd43..51ce2c5 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
@@ -16,19 +16,37 @@
  */
 package org.apache.carbondata.hadoop.test.util;
 
-import com.google.gson.Gson;
-import org.apache.hadoop.fs.Path;
-
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import org.apache.carbondata.common.CarbonIterator;
 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.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
+import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
 import org.apache.carbondata.core.metadata.converter.SchemaConverter;
 import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
 import org.apache.carbondata.core.metadata.datatype.DataType;
@@ -42,14 +60,11 @@ 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.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.impl.FileFactory;
-import org.apache.carbondata.processing.csvreaderstep.BlockDetails;
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonStorePath;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter;
 import org.apache.carbondata.core.writer.CarbonDictionaryWriterImpl;
 import org.apache.carbondata.core.writer.ThriftWriter;
@@ -57,25 +72,29 @@ 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.core.fileoperations.AtomicFileOperations;
-import org.apache.carbondata.core.fileoperations.AtomicFileOperationsImpl;
-import org.apache.carbondata.core.fileoperations.FileWriteOperation;
-import org.apache.carbondata.processing.api.dataloader.DataLoadModel;
 import org.apache.carbondata.processing.api.dataloader.SchemaInfo;
 import org.apache.carbondata.processing.constants.TableOptionConstant;
-import org.apache.carbondata.processing.csvload.DataGraphExecuter;
-import org.apache.carbondata.processing.dataprocessor.DataProcessTaskStatus;
-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.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 java.io.*;
-import java.nio.charset.Charset;
-import java.text.SimpleDateFormat;
-import java.util.*;
+import com.google.gson.Gson;
+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;
+import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 
 /**
  * This class will create store file based on provided schema
+ *
  */
 public class StoreCreator {
 
@@ -104,26 +123,52 @@ public class StoreCreator {
 
     try {
 
-      String factFilePath = new File("src/test/resources/data.csv").getCanonicalPath();
+      String factFilePath = new File("../hadoop/src/test/resources/data.csv").getCanonicalPath();
       File storeDir = new File(absoluteTableIdentifier.getStorePath());
       CarbonUtil.deleteFoldersAndFiles(storeDir);
       CarbonProperties.getInstance().addProperty(CarbonCommonConstants.STORE_LOCATION_HDFS,
           absoluteTableIdentifier.getStorePath());
 
-      String kettleHomePath = "../processing/carbonplugins";
       CarbonTable table = createTable();
       writeDictionary(factFilePath, table);
       CarbonDataLoadSchema schema = new CarbonDataLoadSchema(table);
-      LoadModel loadModel = new LoadModel();
+      CarbonLoadModel loadModel = new CarbonLoadModel();
       String partitionId = "0";
-      loadModel.setSchema(schema);
+      loadModel.setCarbonDataLoadSchema(schema);
       loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
       loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
       loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
       loadModel.setFactFilePath(factFilePath);
       loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
+      loadModel.setStorePath(absoluteTableIdentifier.getStorePath());
+      loadModel.setDateFormat(null);
+      loadModel.setDefaultTimestampFormat(CarbonProperties.getInstance().getProperty(
+          CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+      loadModel.setDefaultDateFormat(CarbonProperties.getInstance().getProperty(
+          CarbonCommonConstants.CARBON_DATE_FORMAT,
+          CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT));
+      loadModel
+          .setSerializationNullFormat(
+              TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N");
+      loadModel
+          .setBadRecordsLoggerEnable(
+              TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false");
+      loadModel
+          .setBadRecordsAction(
+              TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "FORCE");
+      loadModel
+          .setIsEmptyDataBadRecord(
+              DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + "false");
+      loadModel.setCsvHeader("ID,date,country,name,phonetype,serialname,salary");
+      loadModel.setCsvHeaderColumns(loadModel.getCsvHeader().split(","));
+      loadModel.setTaskNo("0");
+      loadModel.setSegmentId("0");
+      loadModel.setPartitionId("0");
+      loadModel.setFactTimeStamp(System.currentTimeMillis());
+
+      executeGraph(loadModel, absoluteTableIdentifier.getStorePath());
 
-      executeGraph(loadModel, absoluteTableIdentifier.getStorePath(), kettleHomePath);
     } catch (Exception e) {
       e.printStackTrace();
     }
@@ -311,12 +356,10 @@ public class StoreCreator {
    *
    * @param loadModel
    * @param storeLocation
-   * @param kettleHomePath
    * @throws Exception
    */
-  public static void executeGraph(LoadModel loadModel, String storeLocation, String kettleHomePath)
+  public static void executeGraph(CarbonLoadModel loadModel, String storeLocation)
       throws Exception {
-    System.setProperty("KETTLE_HOME", kettleHomePath);
     new File(storeLocation).mkdirs();
     String outPutLoc = storeLocation + "/etl";
     String databaseName = loadModel.getDatabaseName();
@@ -344,35 +387,39 @@ public class StoreCreator {
       path.delete();
     }
 
-    DataProcessTaskStatus dataProcessTaskStatus = new DataProcessTaskStatus(databaseName, tableName);
-    dataProcessTaskStatus.setCsvFilePath(loadModel.getFactFilePath());
     SchemaInfo info = new SchemaInfo();
     BlockDetails blockDetails = new BlockDetails(new Path(loadModel.getFactFilePath()),
         0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"});
-    GraphGenerator.blockInfo.put("qwqwq", new BlockDetails[] { blockDetails });
-    dataProcessTaskStatus.setBlocksID("qwqwq");
-    dataProcessTaskStatus.setEscapeCharacter("\\");
-    dataProcessTaskStatus.setQuoteCharacter("\"");
-    dataProcessTaskStatus.setCommentCharacter("#");
-    dataProcessTaskStatus.setDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+    Configuration configuration = new Configuration();
+    CSVInputFormat.setCommentCharacter(configuration, loadModel.getCommentChar());
+    CSVInputFormat.setCSVDelimiter(configuration, loadModel.getCsvDelimiter());
+    CSVInputFormat.setEscapeCharacter(configuration, loadModel.getEscapeChar());
+    CSVInputFormat.setHeaderExtractionEnabled(configuration, true);
+    CSVInputFormat.setQuoteCharacter(configuration, loadModel.getQuoteChar());
+    CSVInputFormat.setReadBufferSize(configuration, CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
+            CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
+
+    TaskAttemptContextImpl hadoopAttemptContext = new TaskAttemptContextImpl(configuration, new TaskAttemptID("", 1, TaskType.MAP, 0, 0));
+    CSVInputFormat format = new CSVInputFormat();
+
+    RecordReader<NullWritable, StringArrayWritable> recordReader =
+        format.createRecordReader(blockDetails, hadoopAttemptContext);
+
+    CSVRecordReaderIterator readerIterator = new CSVRecordReaderIterator(recordReader, blockDetails, hadoopAttemptContext);
+    new DataLoadExecutor().execute(loadModel,
+        storeLocation,
+        new CarbonIterator[]{readerIterator});
+
     info.setDatabaseName(databaseName);
     info.setTableName(tableName);
-    info.setSerializationNullFormat(
-        TableOptionConstant.SERIALIZATION_NULL_FORMAT.getName() + "," + "\\N");
-    info.setBadRecordsLoggerEnable(
-        TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE.getName() + "," + "false");
-    info.setBadRecordsLoggerAction(
-        TableOptionConstant.BAD_RECORDS_ACTION.getName() + "," + "force");
-
-    generateGraph(dataProcessTaskStatus, info, loadModel.getTableName(), "0", loadModel.getSchema(), null,
-        loadModel.getLoadMetadataDetails());
-
-    DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus);
-    graphExecuter
-        .executeGraph(graphPath, info, loadModel.getSchema());
+
+//    DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus);
+//    graphExecuter
+//        .executeGraph(graphPath, info, loadModel.getSchema());
     //    LoadMetadataDetails[] loadDetails =
     //        CarbonUtil.readLoadMetadata(loadModel.schema.getCarbonTable().getMetaDataFilepath());
-    writeLoadMetadata(loadModel.schema, loadModel.getTableName(), loadModel.getTableName(),
+    writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getTableName(), loadModel.getTableName(),
         new ArrayList<LoadMetadataDetails>());
 
     String segLocation =
@@ -403,6 +450,7 @@ public class StoreCreator {
   public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databaseName,
       String tableName, List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
     LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
+    loadMetadataDetails.setLoadEndTime(System.currentTimeMillis());
     loadMetadataDetails.setLoadStatus("SUCCESS");
     loadMetadataDetails.setLoadName(String.valueOf(0));
     loadMetadataDetails.setLoadStartTime(loadMetadataDetails.getTimeStamp(readCurrentTime()));
@@ -442,40 +490,6 @@ public class StoreCreator {
 
   }
 
-  /**
-   * generate graph
-   *
-   * @param dataProcessTaskStatus
-   * @param info
-   * @param tableName
-   * @param partitionID
-   * @param schema
-   * @param factStoreLocation
-   * @param loadMetadataDetails
-   * @throws GraphGeneratorException
-   */
-  private static void generateGraph(IDataProcessStatus dataProcessTaskStatus, SchemaInfo info,
-      String tableName, String partitionID, CarbonDataLoadSchema schema, String factStoreLocation,
-      List<LoadMetadataDetails> loadMetadataDetails)
-      throws GraphGeneratorException {
-    DataLoadModel model = new DataLoadModel();
-    model.setCsvLoad(null != dataProcessTaskStatus.getCsvFilePath() || null != dataProcessTaskStatus.getFilesToProcess());
-    model.setSchemaInfo(info);
-    model.setTableName(dataProcessTaskStatus.getTableName());
-    model.setTaskNo("1");
-    model.setBlocksID(dataProcessTaskStatus.getBlocksID());
-    model.setFactTimeStamp(System.currentTimeMillis() + "");
-    model.setEscapeCharacter(dataProcessTaskStatus.getEscapeCharacter());
-    model.setQuoteCharacter(dataProcessTaskStatus.getQuoteCharacter());
-    model.setCommentCharacter(dataProcessTaskStatus.getCommentCharacter());
-    model.setDateFormat(dataProcessTaskStatus.getDateFormat());
-    String outputLocation = CarbonProperties.getInstance()
-        .getProperty("store_output_location", "../carbon-store/system/carbon/etl");
-    GraphGenerator generator =
-        new GraphGenerator(model, partitionID, factStoreLocation, schema, "0", outputLocation);
-    generator.generateGraph();
-  }
-
   public static String readCurrentTime() {
     SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
     String date = null;
@@ -485,60 +499,6 @@ public class StoreCreator {
     return date;
   }
 
-  /**
-   * This is local model object used inside this class to store information related to data loading
-   */
-  private static class LoadModel {
-
-    private CarbonDataLoadSchema schema;
-    private String tableName;
-    private String databaseName;
-    private List<LoadMetadataDetails> loadMetaDetail;
-    private String factFilePath;
-
-    public void setSchema(CarbonDataLoadSchema schema) {
-      this.schema = schema;
-    }
-
-    public List<LoadMetadataDetails> getLoadMetadataDetails() {
-      return loadMetaDetail;
-    }
-
-    public CarbonDataLoadSchema getSchema() {
-      return schema;
-    }
-
-    public String getFactFilePath() {
-      return factFilePath;
-    }
-
-    public String getTableName() {
-      return tableName;
-    }
-
-    public String getDatabaseName() {
-      return databaseName;
-    }
-
-    public void setLoadMetadataDetails(List<LoadMetadataDetails> loadMetaDetail) {
-      this.loadMetaDetail = loadMetaDetail;
-    }
-
-    public void setFactFilePath(String factFilePath) {
-      this.factFilePath = factFilePath;
-    }
-
-    public void setTableName(String tableName) {
-      this.tableName = tableName;
-    }
-
-    public void setDatabaseName(String databaseName) {
-      this.databaseName = databaseName;
-    }
-
-
-  }
-
   public static void main(String[] args) {
     StoreCreator.createCarbonStore();
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common-test/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/pom.xml b/integration/spark-common-test/pom.xml
index d92bc94..f9bfaa0 100644
--- a/integration/spark-common-test/pom.xml
+++ b/integration/spark-common-test/pom.xml
@@ -147,7 +147,6 @@
           </environmentVariables>
           <systemProperties>
             <java.awt.headless>true</java.awt.headless>
-            <use_kettle>${use.kettle}</use_kettle>
           </systemProperties>
         </configuration>
         <executions>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
index aa18b8f..58b49d4 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestLoadDataGeneral.scala
@@ -136,7 +136,7 @@ class TestLoadDataGeneral extends QueryTest with BeforeAndAfterAll {
         STORED BY 'org.apache.carbondata.format' """)
     val testData = s"$resourcesPath/sample.csv"
     try {
-      sql(s"LOAD DATA LOCAL INPATH '$testData' into table load_test_singlepass options ('USE_KETTLE'='FALSE','SINGLE_PASS'='TRUE')")
+      sql(s"LOAD DATA LOCAL INPATH '$testData' into table load_test_singlepass options ('SINGLE_PASS'='TRUE')")
     } catch {
       case ex: Exception =>
         assert(false)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala
index 052ee45..7df7fcd 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/DoubleDataTypeTest.scala
@@ -25,7 +25,7 @@ class DoubleDataTypeTest extends QueryTest with BeforeAndAfterAll {
   override def beforeAll: Unit = {
     sql("DROP TABLE IF EXISTS oscon_carbon_old")
     sql("""create table oscon_carbon_old (CUST_PRFRD_FLG String,PROD_BRAND_NAME String,PROD_COLOR String,CUST_LAST_RVW_DATE String,CUST_COUNTRY String,CUST_CITY String,PRODUCT_NAME String,CUST_JOB_TITLE String,CUST_STATE String,CUST_BUY_POTENTIAL String,PRODUCT_MODEL String,ITM_ID String,ITM_NAME String,PRMTION_ID String,PRMTION_NAME String,SHP_MODE_ID String,SHP_MODE String,DELIVERY_COUNTRY String,DELIVERY_STATE String,DELIVERY_CITY String,DELIVERY_DISTRICT String,ACTIVE_EMUI_VERSION String,WH_NAME String,STR_ORDER_DATE String,OL_ORDER_NO String,OL_ORDER_DATE String,OL_SITE String,CUST_FIRST_NAME String,CUST_LAST_NAME String,CUST_BIRTH_DY String,CUST_BIRTH_MM String,CUST_BIRTH_YR String,CUST_BIRTH_COUNTRY String,CUST_SEX String,CUST_ADDRESS_ID String,CUST_STREET_NO String,CUST_STREET_NAME String,CUST_AGE String,CUST_SUITE_NO String,CUST_ZIP String,CUST_COUNTY String,PRODUCT_ID String,PROD_SHELL_COLOR String,DEVICE_NAME String,PROD_SHORT_DESC String,PROD_LONG_DESC String,PROD_THUMB 
 String,PROD_IMAGE String,PROD_UPDATE_DATE String,PROD_LIVE String,PROD_LOC String,PROD_RAM String,PROD_ROM String,PROD_CPU_CLOCK String,PROD_SERIES String,ITM_REC_START_DATE String,ITM_REC_END_DATE String,ITM_BRAND_ID String,ITM_BRAND String,ITM_CLASS_ID String,ITM_CLASS String,ITM_CATEGORY_ID String,ITM_CATEGORY String,ITM_MANUFACT_ID String,ITM_MANUFACT String,ITM_FORMULATION String,ITM_COLOR String,ITM_CONTAINER String,ITM_MANAGER_ID String,PRM_START_DATE String,PRM_END_DATE String,PRM_CHANNEL_DMAIL String,PRM_CHANNEL_EMAIL String,PRM_CHANNEL_CAT String,PRM_CHANNEL_TV String,PRM_CHANNEL_RADIO String,PRM_CHANNEL_PRESS String,PRM_CHANNEL_EVENT String,PRM_CHANNEL_DEMO String,PRM_CHANNEL_DETAILS String,PRM_PURPOSE String,PRM_DSCNT_ACTIVE String,SHP_CODE String,SHP_CARRIER String,SHP_CONTRACT String,CHECK_DATE String,CHECK_YR String,CHECK_MM String,CHECK_DY String,CHECK_HOUR String,BOM String,INSIDE_NAME String,PACKING_DATE String,PACKING_YR String,PACKING_MM String,PACKING_DY String,
 PACKING_HOUR String,DELIVERY_PROVINCE String,PACKING_LIST_NO String,ACTIVE_CHECK_TIME String,ACTIVE_CHECK_YR String,ACTIVE_CHECK_MM String,ACTIVE_CHECK_DY String,ACTIVE_CHECK_HOUR String,ACTIVE_AREA_ID String,ACTIVE_COUNTRY String,ACTIVE_PROVINCE String,ACTIVE_CITY String,ACTIVE_DISTRICT String,ACTIVE_NETWORK String,ACTIVE_FIRMWARE_VER String,ACTIVE_OS_VERSION String,LATEST_CHECK_TIME String,LATEST_CHECK_YR String,LATEST_CHECK_MM String,LATEST_CHECK_DY String,LATEST_CHECK_HOUR String,LATEST_AREAID String,LATEST_COUNTRY String,LATEST_PROVINCE String,LATEST_CITY String,LATEST_DISTRICT String,LATEST_FIRMWARE_VER String,LATEST_EMUI_VERSION String,LATEST_OS_VERSION String,LATEST_NETWORK String,WH_ID String,WH_STREET_NO String,WH_STREET_NAME String,WH_STREET_TYPE String,WH_SUITE_NO String,WH_CITY String,WH_COUNTY String,WH_STATE String,WH_ZIP String,WH_COUNTRY String,OL_SITE_DESC String,OL_RET_ORDER_NO String,OL_RET_DATE String,PROD_MODEL_ID String,CUST_ID String,PROD_UNQ_MDL_ID String,CU
 ST_NICK_NAME String,CUST_LOGIN String,CUST_EMAIL_ADDR String,PROD_UNQ_DEVICE_ADDR String,PROD_UQ_UUID String,PROD_BAR_CODE String,TRACKING_NO String,STR_ORDER_NO String,CUST_DEP_COUNT double,CUST_VEHICLE_COUNT double,CUST_ADDRESS_CNT double,CUST_CRNT_CDEMO_CNT double,CUST_CRNT_HDEMO_CNT double,CUST_CRNT_ADDR_DM double,CUST_FIRST_SHIPTO_CNT double,CUST_FIRST_SALES_CNT double,CUST_GMT_OFFSET double,CUST_DEMO_CNT double,CUST_INCOME double,PROD_UNLIMITED double,PROD_OFF_PRICE double,PROD_UNITS double,TOTAL_PRD_COST double,TOTAL_PRD_DISC double,PROD_WEIGHT double,REG_UNIT_PRICE double,EXTENDED_AMT double,UNIT_PRICE_DSCNT_PCT double,DSCNT_AMT double,PROD_STD_CST double,TOTAL_TX_AMT double,FREIGHT_CHRG double,WAITING_PERIOD double,DELIVERY_PERIOD double,ITM_CRNT_PRICE double,ITM_UNITS double,ITM_WSLE_CST double,ITM_SIZE double,PRM_CST double,PRM_RESPONSE_TARGET double,PRM_ITM_DM double,SHP_MODE_CNT double,WH_GMT_OFFSET double,WH_SQ_FT double,STR_ORD_QTY double,STR_WSLE_CST double,STR_LIST_
 PRICE double,STR_SALES_PRICE double,STR_EXT_DSCNT_AMT double,STR_EXT_SALES_PRICE double,STR_EXT_WSLE_CST double,STR_EXT_LIST_PRICE double,STR_EXT_TX double,STR_COUPON_AMT double,STR_NET_PAID double,STR_NET_PAID_INC_TX double,STR_NET_PRFT double,STR_SOLD_YR_CNT double,STR_SOLD_MM_CNT double,STR_SOLD_ITM_CNT double,STR_TOTAL_CUST_CNT double,STR_AREA_CNT double,STR_DEMO_CNT double,STR_OFFER_CNT double,STR_PRM_CNT double,STR_TICKET_CNT double,STR_NET_PRFT_DM_A double,STR_NET_PRFT_DM_B double,STR_NET_PRFT_DM_C double,STR_NET_PRFT_DM_D double,STR_NET_PRFT_DM_E double,STR_RET_STR_ID double,STR_RET_REASON_CNT double,STR_RET_TICKET_NO double,STR_RTRN_QTY double,STR_RTRN_AMT double,STR_RTRN_TX double,STR_RTRN_AMT_INC_TX double,STR_RET_FEE double,STR_RTRN_SHIP_CST double,STR_RFNDD_CSH double,STR_REVERSED_CHRG double,STR_STR_CREDIT double,STR_RET_NET_LOSS double,STR_RTRNED_YR_CNT double,STR_RTRN_MM_CNT double,STR_RET_ITM_CNT double,STR_RET_CUST_CNT double,STR_RET_AREA_CNT double,STR_RET_OFFER_C
 NT double,STR_RET_PRM_CNT double,STR_RET_NET_LOSS_DM_A double,STR_RET_NET_LOSS_DM_B double,STR_RET_NET_LOSS_DM_C double,STR_RET_NET_LOSS_DM_D double,OL_ORD_QTY double,OL_WSLE_CST double,OL_LIST_PRICE double,OL_SALES_PRICE double,OL_EXT_DSCNT_AMT double,OL_EXT_SALES_PRICE double,OL_EXT_WSLE_CST double,OL_EXT_LIST_PRICE double,OL_EXT_TX double,OL_COUPON_AMT double,OL_EXT_SHIP_CST double,OL_NET_PAID double,OL_NET_PAID_INC_TX double,OL_NET_PAID_INC_SHIP double,OL_NET_PAID_INC_SHIP_TX double,OL_NET_PRFT double,OL_SOLD_YR_CNT double,OL_SOLD_MM_CNT double,OL_SHIP_DATE_CNT double,OL_ITM_CNT double,OL_BILL_CUST_CNT double,OL_BILL_AREA_CNT double,OL_BILL_DEMO_CNT double,OL_BILL_OFFER_CNT double,OL_SHIP_CUST_CNT double,OL_SHIP_AREA_CNT double,OL_SHIP_DEMO_CNT double,OL_SHIP_OFFER_CNT double,OL_WEB_PAGE_CNT double,OL_WEB_SITE_CNT double,OL_SHIP_MODE_CNT double,OL_WH_CNT double,OL_PRM_CNT double,OL_NET_PRFT_DM_A double,OL_NET_PRFT_DM_B double,OL_NET_PRFT_DM_C double,OL_NET_PRFT_DM_D double,OL_RE
 T_RTRN_QTY double,OL_RTRN_AMT double,OL_RTRN_TX double,OL_RTRN_AMT_INC_TX double,OL_RET_FEE double,OL_RTRN_SHIP_CST double,OL_RFNDD_CSH double,OL_REVERSED_CHRG double,OL_ACCOUNT_CREDIT double,OL_RTRNED_YR_CNT double,OL_RTRNED_MM_CNT double,OL_RTRITM_CNT double,OL_RFNDD_CUST_CNT double,OL_RFNDD_AREA_CNT double,OL_RFNDD_DEMO_CNT double,OL_RFNDD_OFFER_CNT double,OL_RTRNING_CUST_CNT double,OL_RTRNING_AREA_CNT double,OL_RTRNING_DEMO_CNT double,OL_RTRNING_OFFER_CNT double,OL_RTRWEB_PAGE_CNT double,OL_REASON_CNT double,OL_NET_LOSS double,OL_NET_LOSS_DM_A double,OL_NET_LOSS_DM_B double,OL_NET_LOSS_DM_C double) STORED BY 'org.apache.carbondata.format' tblproperties('DICTIONARY_EXCLUDE'='CUST_ID,CUST_NICK_NAME,CUST_FIRST_NAME,CUST_LAST_NAME,CUST_LOGIN,CUST_EMAIL_ADDR,PROD_UNQ_MDL_ID,PROD_UNQ_DEVICE_ADDR,PROD_UQ_UUID,DEVICE_NAME,PROD_BAR_CODE,ITM_ID,ITM_NAME,ITM_BRAND_ID,ITM_BRAND,BOM,PACKING_LIST_NO,TRACKING_NO,ACTIVE_FIRMWARE_VER,LATEST_FIRMWARE_VER,LATEST_EMUI_VERSION,LATEST_NETWORK,STR_ORD
 ER_NO','table_blocksize'='256')""")
-    sql(s"""load data LOCAL inpath '$resourcesPath/oscon_10.csv' into table oscon_carbon_old options('USE_KETTLE'='true','DELIMITER'=',', 'QUOTECHAR'='\"','FILEHEADER'='ACTIVE_AREA_ID, ACTIVE_CHECK_DY, ACTIVE_CHECK_HOUR, ACTIVE_CHECK_MM, ACTIVE_CHECK_TIME, ACTIVE_CHECK_YR, ACTIVE_CITY, ACTIVE_COUNTRY, ACTIVE_DISTRICT, ACTIVE_EMUI_VERSION, ACTIVE_FIRMWARE_VER, ACTIVE_NETWORK, ACTIVE_OS_VERSION, ACTIVE_PROVINCE, BOM, CHECK_DATE, CHECK_DY, CHECK_HOUR, CHECK_MM, CHECK_YR, CUST_ADDRESS_ID, CUST_AGE, CUST_BIRTH_COUNTRY, CUST_BIRTH_DY, CUST_BIRTH_MM, CUST_BIRTH_YR, CUST_BUY_POTENTIAL, CUST_CITY, CUST_STATE, CUST_COUNTRY, CUST_COUNTY, CUST_EMAIL_ADDR, CUST_LAST_RVW_DATE, CUST_FIRST_NAME, CUST_ID, CUST_JOB_TITLE, CUST_LAST_NAME, CUST_LOGIN, CUST_NICK_NAME, CUST_PRFRD_FLG, CUST_SEX, CUST_STREET_NAME, CUST_STREET_NO, CUST_SUITE_NO, CUST_ZIP, DELIVERY_CITY, DELIVERY_STATE, DELIVERY_COUNTRY, DELIVERY_DISTRICT, DELIVERY_PROVINCE, DEVICE_NAME, INSIDE_NAME, ITM_BRAND, ITM_BRAND_ID, ITM_CATEGORY, IT
 M_CATEGORY_ID, ITM_CLASS, ITM_CLASS_ID, ITM_COLOR, ITM_CONTAINER, ITM_FORMULATION, ITM_MANAGER_ID, ITM_MANUFACT, ITM_MANUFACT_ID, ITM_ID, ITM_NAME, ITM_REC_END_DATE, ITM_REC_START_DATE, LATEST_AREAID, LATEST_CHECK_DY, LATEST_CHECK_HOUR, LATEST_CHECK_MM, LATEST_CHECK_TIME, LATEST_CHECK_YR, LATEST_CITY, LATEST_COUNTRY, LATEST_DISTRICT, LATEST_EMUI_VERSION, LATEST_FIRMWARE_VER, LATEST_NETWORK, LATEST_OS_VERSION, LATEST_PROVINCE, OL_ORDER_DATE, OL_ORDER_NO, OL_RET_ORDER_NO, OL_RET_DATE, OL_SITE, OL_SITE_DESC, PACKING_DATE, PACKING_DY, PACKING_HOUR, PACKING_LIST_NO, PACKING_MM, PACKING_YR, PRMTION_ID, PRMTION_NAME, PRM_CHANNEL_CAT, PRM_CHANNEL_DEMO, PRM_CHANNEL_DETAILS, PRM_CHANNEL_DMAIL, PRM_CHANNEL_EMAIL, PRM_CHANNEL_EVENT, PRM_CHANNEL_PRESS, PRM_CHANNEL_RADIO, PRM_CHANNEL_TV, PRM_DSCNT_ACTIVE, PRM_END_DATE, PRM_PURPOSE, PRM_START_DATE, PRODUCT_ID, PROD_BAR_CODE, PROD_BRAND_NAME, PRODUCT_NAME, PRODUCT_MODEL, PROD_MODEL_ID, PROD_COLOR, PROD_SHELL_COLOR, PROD_CPU_CLOCK, PROD_IMAGE, PROD_
 LIVE, PROD_LOC, PROD_LONG_DESC, PROD_RAM, PROD_ROM, PROD_SERIES, PROD_SHORT_DESC, PROD_THUMB, PROD_UNQ_DEVICE_ADDR, PROD_UNQ_MDL_ID, PROD_UPDATE_DATE, PROD_UQ_UUID, SHP_CARRIER, SHP_CODE, SHP_CONTRACT, SHP_MODE_ID, SHP_MODE, STR_ORDER_DATE, STR_ORDER_NO, TRACKING_NO, WH_CITY, WH_COUNTRY, WH_COUNTY, WH_ID, WH_NAME, WH_STATE, WH_STREET_NAME, WH_STREET_NO, WH_STREET_TYPE, WH_SUITE_NO, WH_ZIP, CUST_DEP_COUNT, CUST_VEHICLE_COUNT, CUST_ADDRESS_CNT, CUST_CRNT_CDEMO_CNT, CUST_CRNT_HDEMO_CNT, CUST_CRNT_ADDR_DM, CUST_FIRST_SHIPTO_CNT, CUST_FIRST_SALES_CNT, CUST_GMT_OFFSET, CUST_DEMO_CNT, CUST_INCOME, PROD_UNLIMITED, PROD_OFF_PRICE, PROD_UNITS, TOTAL_PRD_COST, TOTAL_PRD_DISC, PROD_WEIGHT, REG_UNIT_PRICE, EXTENDED_AMT, UNIT_PRICE_DSCNT_PCT, DSCNT_AMT, PROD_STD_CST, TOTAL_TX_AMT, FREIGHT_CHRG, WAITING_PERIOD, DELIVERY_PERIOD, ITM_CRNT_PRICE, ITM_UNITS, ITM_WSLE_CST, ITM_SIZE, PRM_CST, PRM_RESPONSE_TARGET, PRM_ITM_DM, SHP_MODE_CNT, WH_GMT_OFFSET, WH_SQ_FT, STR_ORD_QTY, STR_WSLE_CST, STR_LIST_PRIC
 E, STR_SALES_PRICE, STR_EXT_DSCNT_AMT, STR_EXT_SALES_PRICE, STR_EXT_WSLE_CST, STR_EXT_LIST_PRICE, STR_EXT_TX, STR_COUPON_AMT, STR_NET_PAID, STR_NET_PAID_INC_TX, STR_NET_PRFT, STR_SOLD_YR_CNT, STR_SOLD_MM_CNT, STR_SOLD_ITM_CNT, STR_TOTAL_CUST_CNT, STR_AREA_CNT, STR_DEMO_CNT, STR_OFFER_CNT, STR_PRM_CNT, STR_TICKET_CNT, STR_NET_PRFT_DM_A, STR_NET_PRFT_DM_B, STR_NET_PRFT_DM_C, STR_NET_PRFT_DM_D, STR_NET_PRFT_DM_E, STR_RET_STR_ID, STR_RET_REASON_CNT, STR_RET_TICKET_NO, STR_RTRN_QTY, STR_RTRN_AMT, STR_RTRN_TX, STR_RTRN_AMT_INC_TX, STR_RET_FEE, STR_RTRN_SHIP_CST, STR_RFNDD_CSH, STR_REVERSED_CHRG, STR_STR_CREDIT, STR_RET_NET_LOSS, STR_RTRNED_YR_CNT, STR_RTRN_MM_CNT, STR_RET_ITM_CNT, STR_RET_CUST_CNT, STR_RET_AREA_CNT, STR_RET_OFFER_CNT, STR_RET_PRM_CNT, STR_RET_NET_LOSS_DM_A, STR_RET_NET_LOSS_DM_B, STR_RET_NET_LOSS_DM_C, STR_RET_NET_LOSS_DM_D, OL_ORD_QTY, OL_WSLE_CST, OL_LIST_PRICE, OL_SALES_PRICE, OL_EXT_DSCNT_AMT, OL_EXT_SALES_PRICE, OL_EXT_WSLE_CST, OL_EXT_LIST_PRICE, OL_EXT_TX, OL_COUPO
 N_AMT, OL_EXT_SHIP_CST, OL_NET_PAID, OL_NET_PAID_INC_TX, OL_NET_PAID_INC_SHIP, OL_NET_PAID_INC_SHIP_TX, OL_NET_PRFT, OL_SOLD_YR_CNT, OL_SOLD_MM_CNT, OL_SHIP_DATE_CNT, OL_ITM_CNT, OL_BILL_CUST_CNT, OL_BILL_AREA_CNT, OL_BILL_DEMO_CNT, OL_BILL_OFFER_CNT, OL_SHIP_CUST_CNT, OL_SHIP_AREA_CNT, OL_SHIP_DEMO_CNT, OL_SHIP_OFFER_CNT, OL_WEB_PAGE_CNT, OL_WEB_SITE_CNT, OL_SHIP_MODE_CNT, OL_WH_CNT, OL_PRM_CNT, OL_NET_PRFT_DM_A, OL_NET_PRFT_DM_B, OL_NET_PRFT_DM_C, OL_NET_PRFT_DM_D, OL_RET_RTRN_QTY, OL_RTRN_AMT, OL_RTRN_TX, OL_RTRN_AMT_INC_TX, OL_RET_FEE, OL_RTRN_SHIP_CST, OL_RFNDD_CSH, OL_REVERSED_CHRG, OL_ACCOUNT_CREDIT, OL_RTRNED_YR_CNT, OL_RTRNED_MM_CNT, OL_RTRITM_CNT, OL_RFNDD_CUST_CNT, OL_RFNDD_AREA_CNT, OL_RFNDD_DEMO_CNT, OL_RFNDD_OFFER_CNT, OL_RTRNING_CUST_CNT, OL_RTRNING_AREA_CNT, OL_RTRNING_DEMO_CNT, OL_RTRNING_OFFER_CNT, OL_RTRWEB_PAGE_CNT, OL_REASON_CNT, OL_NET_LOSS, OL_NET_LOSS_DM_A, OL_NET_LOSS_DM_B, OL_NET_LOSS_DM_C','BAD_RECORDS_ACTION'='FORCE','BAD_RECORDS_LOGGER_ENABLE'='FALSE')""
 ")
+    sql(s"""load data LOCAL inpath '$resourcesPath/oscon_10.csv' into table oscon_carbon_old options('DELIMITER'=',', 'QUOTECHAR'='\"','FILEHEADER'='ACTIVE_AREA_ID, ACTIVE_CHECK_DY, ACTIVE_CHECK_HOUR, ACTIVE_CHECK_MM, ACTIVE_CHECK_TIME, ACTIVE_CHECK_YR, ACTIVE_CITY, ACTIVE_COUNTRY, ACTIVE_DISTRICT, ACTIVE_EMUI_VERSION, ACTIVE_FIRMWARE_VER, ACTIVE_NETWORK, ACTIVE_OS_VERSION, ACTIVE_PROVINCE, BOM, CHECK_DATE, CHECK_DY, CHECK_HOUR, CHECK_MM, CHECK_YR, CUST_ADDRESS_ID, CUST_AGE, CUST_BIRTH_COUNTRY, CUST_BIRTH_DY, CUST_BIRTH_MM, CUST_BIRTH_YR, CUST_BUY_POTENTIAL, CUST_CITY, CUST_STATE, CUST_COUNTRY, CUST_COUNTY, CUST_EMAIL_ADDR, CUST_LAST_RVW_DATE, CUST_FIRST_NAME, CUST_ID, CUST_JOB_TITLE, CUST_LAST_NAME, CUST_LOGIN, CUST_NICK_NAME, CUST_PRFRD_FLG, CUST_SEX, CUST_STREET_NAME, CUST_STREET_NO, CUST_SUITE_NO, CUST_ZIP, DELIVERY_CITY, DELIVERY_STATE, DELIVERY_COUNTRY, DELIVERY_DISTRICT, DELIVERY_PROVINCE, DEVICE_NAME, INSIDE_NAME, ITM_BRAND, ITM_BRAND_ID, ITM_CATEGORY, ITM_CATEGORY_ID, ITM_C
 LASS, ITM_CLASS_ID, ITM_COLOR, ITM_CONTAINER, ITM_FORMULATION, ITM_MANAGER_ID, ITM_MANUFACT, ITM_MANUFACT_ID, ITM_ID, ITM_NAME, ITM_REC_END_DATE, ITM_REC_START_DATE, LATEST_AREAID, LATEST_CHECK_DY, LATEST_CHECK_HOUR, LATEST_CHECK_MM, LATEST_CHECK_TIME, LATEST_CHECK_YR, LATEST_CITY, LATEST_COUNTRY, LATEST_DISTRICT, LATEST_EMUI_VERSION, LATEST_FIRMWARE_VER, LATEST_NETWORK, LATEST_OS_VERSION, LATEST_PROVINCE, OL_ORDER_DATE, OL_ORDER_NO, OL_RET_ORDER_NO, OL_RET_DATE, OL_SITE, OL_SITE_DESC, PACKING_DATE, PACKING_DY, PACKING_HOUR, PACKING_LIST_NO, PACKING_MM, PACKING_YR, PRMTION_ID, PRMTION_NAME, PRM_CHANNEL_CAT, PRM_CHANNEL_DEMO, PRM_CHANNEL_DETAILS, PRM_CHANNEL_DMAIL, PRM_CHANNEL_EMAIL, PRM_CHANNEL_EVENT, PRM_CHANNEL_PRESS, PRM_CHANNEL_RADIO, PRM_CHANNEL_TV, PRM_DSCNT_ACTIVE, PRM_END_DATE, PRM_PURPOSE, PRM_START_DATE, PRODUCT_ID, PROD_BAR_CODE, PROD_BRAND_NAME, PRODUCT_NAME, PRODUCT_MODEL, PROD_MODEL_ID, PROD_COLOR, PROD_SHELL_COLOR, PROD_CPU_CLOCK, PROD_IMAGE, PROD_LIVE, PROD_LOC, PROD
 _LONG_DESC, PROD_RAM, PROD_ROM, PROD_SERIES, PROD_SHORT_DESC, PROD_THUMB, PROD_UNQ_DEVICE_ADDR, PROD_UNQ_MDL_ID, PROD_UPDATE_DATE, PROD_UQ_UUID, SHP_CARRIER, SHP_CODE, SHP_CONTRACT, SHP_MODE_ID, SHP_MODE, STR_ORDER_DATE, STR_ORDER_NO, TRACKING_NO, WH_CITY, WH_COUNTRY, WH_COUNTY, WH_ID, WH_NAME, WH_STATE, WH_STREET_NAME, WH_STREET_NO, WH_STREET_TYPE, WH_SUITE_NO, WH_ZIP, CUST_DEP_COUNT, CUST_VEHICLE_COUNT, CUST_ADDRESS_CNT, CUST_CRNT_CDEMO_CNT, CUST_CRNT_HDEMO_CNT, CUST_CRNT_ADDR_DM, CUST_FIRST_SHIPTO_CNT, CUST_FIRST_SALES_CNT, CUST_GMT_OFFSET, CUST_DEMO_CNT, CUST_INCOME, PROD_UNLIMITED, PROD_OFF_PRICE, PROD_UNITS, TOTAL_PRD_COST, TOTAL_PRD_DISC, PROD_WEIGHT, REG_UNIT_PRICE, EXTENDED_AMT, UNIT_PRICE_DSCNT_PCT, DSCNT_AMT, PROD_STD_CST, TOTAL_TX_AMT, FREIGHT_CHRG, WAITING_PERIOD, DELIVERY_PERIOD, ITM_CRNT_PRICE, ITM_UNITS, ITM_WSLE_CST, ITM_SIZE, PRM_CST, PRM_RESPONSE_TARGET, PRM_ITM_DM, SHP_MODE_CNT, WH_GMT_OFFSET, WH_SQ_FT, STR_ORD_QTY, STR_WSLE_CST, STR_LIST_PRICE, STR_SALES_PRICE, 
 STR_EXT_DSCNT_AMT, STR_EXT_SALES_PRICE, STR_EXT_WSLE_CST, STR_EXT_LIST_PRICE, STR_EXT_TX, STR_COUPON_AMT, STR_NET_PAID, STR_NET_PAID_INC_TX, STR_NET_PRFT, STR_SOLD_YR_CNT, STR_SOLD_MM_CNT, STR_SOLD_ITM_CNT, STR_TOTAL_CUST_CNT, STR_AREA_CNT, STR_DEMO_CNT, STR_OFFER_CNT, STR_PRM_CNT, STR_TICKET_CNT, STR_NET_PRFT_DM_A, STR_NET_PRFT_DM_B, STR_NET_PRFT_DM_C, STR_NET_PRFT_DM_D, STR_NET_PRFT_DM_E, STR_RET_STR_ID, STR_RET_REASON_CNT, STR_RET_TICKET_NO, STR_RTRN_QTY, STR_RTRN_AMT, STR_RTRN_TX, STR_RTRN_AMT_INC_TX, STR_RET_FEE, STR_RTRN_SHIP_CST, STR_RFNDD_CSH, STR_REVERSED_CHRG, STR_STR_CREDIT, STR_RET_NET_LOSS, STR_RTRNED_YR_CNT, STR_RTRN_MM_CNT, STR_RET_ITM_CNT, STR_RET_CUST_CNT, STR_RET_AREA_CNT, STR_RET_OFFER_CNT, STR_RET_PRM_CNT, STR_RET_NET_LOSS_DM_A, STR_RET_NET_LOSS_DM_B, STR_RET_NET_LOSS_DM_C, STR_RET_NET_LOSS_DM_D, OL_ORD_QTY, OL_WSLE_CST, OL_LIST_PRICE, OL_SALES_PRICE, OL_EXT_DSCNT_AMT, OL_EXT_SALES_PRICE, OL_EXT_WSLE_CST, OL_EXT_LIST_PRICE, OL_EXT_TX, OL_COUPON_AMT, OL_EXT_SHIP_C
 ST, OL_NET_PAID, OL_NET_PAID_INC_TX, OL_NET_PAID_INC_SHIP, OL_NET_PAID_INC_SHIP_TX, OL_NET_PRFT, OL_SOLD_YR_CNT, OL_SOLD_MM_CNT, OL_SHIP_DATE_CNT, OL_ITM_CNT, OL_BILL_CUST_CNT, OL_BILL_AREA_CNT, OL_BILL_DEMO_CNT, OL_BILL_OFFER_CNT, OL_SHIP_CUST_CNT, OL_SHIP_AREA_CNT, OL_SHIP_DEMO_CNT, OL_SHIP_OFFER_CNT, OL_WEB_PAGE_CNT, OL_WEB_SITE_CNT, OL_SHIP_MODE_CNT, OL_WH_CNT, OL_PRM_CNT, OL_NET_PRFT_DM_A, OL_NET_PRFT_DM_B, OL_NET_PRFT_DM_C, OL_NET_PRFT_DM_D, OL_RET_RTRN_QTY, OL_RTRN_AMT, OL_RTRN_TX, OL_RTRN_AMT_INC_TX, OL_RET_FEE, OL_RTRN_SHIP_CST, OL_RFNDD_CSH, OL_REVERSED_CHRG, OL_ACCOUNT_CREDIT, OL_RTRNED_YR_CNT, OL_RTRNED_MM_CNT, OL_RTRITM_CNT, OL_RFNDD_CUST_CNT, OL_RFNDD_AREA_CNT, OL_RFNDD_DEMO_CNT, OL_RFNDD_OFFER_CNT, OL_RTRNING_CUST_CNT, OL_RTRNING_AREA_CNT, OL_RTRNING_DEMO_CNT, OL_RTRNING_OFFER_CNT, OL_RTRWEB_PAGE_CNT, OL_REASON_CNT, OL_NET_LOSS, OL_NET_LOSS_DM_A, OL_NET_LOSS_DM_B, OL_NET_LOSS_DM_C','BAD_RECORDS_ACTION'='FORCE','BAD_RECORDS_LOGGER_ENABLE'='FALSE')""")
   }
 
   test("test to check result for double data type") {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
index 8670d8f..353db9e 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
@@ -93,8 +93,7 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
         "row format delimited fields terminated by ','"
     )
 
-    sql(s"LOAD DATA local inpath '$resourcesPath/data.csv' INTO table smallinttable " +
-      "OPTIONS('USE_KETTLE'='false')")
+    sql(s"LOAD DATA local inpath '$resourcesPath/data.csv' INTO table smallinttable ")
     sql(s"LOAD DATA local inpath '$resourcesPath/datawithoutheader.csv' overwrite " +
       "INTO table smallinthivetable")
 


[04/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java
deleted file mode 100644
index 2d85afa..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java
+++ /dev/null
@@ -1,1436 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
-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.schema.metadata.ColumnSchemaDetailsWrapper;
-import org.apache.carbondata.processing.schema.metadata.HierarchiesInfo;
-import org.apache.carbondata.processing.schema.metadata.TableOptionWrapper;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-import org.apache.carbondata.processing.util.NonDictionaryUtil;
-
-import org.pentaho.di.core.CheckResultInterface;
-import org.pentaho.di.core.Counter;
-import org.pentaho.di.core.database.DatabaseMeta;
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.exception.KettleValueException;
-import org.pentaho.di.core.exception.KettleXMLException;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.core.xml.XMLHandler;
-import org.pentaho.di.i18n.BaseMessages;
-import org.pentaho.di.repository.ObjectId;
-import org.pentaho.di.repository.Repository;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStepMeta;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-import org.w3c.dom.Node;
-
-public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaInterface {
-
-  /**
-   * pkg
-   */
-  private static Class<?> pkg = CarbonCSVBasedSeqGenMeta.class;
-  /**
-   * Foreign key and respective hierarchy Map
-   */
-  protected Map<String, String> foreignKeyHierarchyMap;
-  /**
-   * hier name
-   */
-  protected String[] hierNames;
-  /**
-   * dims
-   */
-  protected int[] dims;
-  /**
-   * dims
-   */
-  protected Map<String, GenericDataType> complexTypes =
-      new HashMap<String, GenericDataType>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  protected Map<String, Map<String, String>> columnProperties =
-      new HashMap<String, Map<String, String>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  /**
-   * dimLens
-   */
-  protected int[] dimLens;
-  /**
-   * msrs
-   */
-  protected int[] msrs;
-  /**
-   * hirches
-   */
-  protected Map<String, int[]> hirches;
-  /**
-   * Hierarchy Column Name map
-   */
-  protected Map<String, String[]> hierColumnMap;
-  /**
-   * dateFormat
-   */
-  protected String dateFormat;
-  /**
-   * timeDimeIndex
-   */
-  protected int timeDimeIndex = -1;
-  /**
-   * timeOrdinalCols
-   */
-  protected String[] timeOrdinalCols = new String[0];
-  /**
-   * timeOrdinalIndices
-   */
-  protected int[] timeOrdinalIndices = new int[0];
-  /**
-   * dimColNames
-   */
-  protected String[] dimColNames;
-  /**
-   *
-   */
-  protected String[] noDictionaryCols;
-  /**
-   *
-   */
-  protected Map<String, String> dimColDataTypes;
-  /**
-   * measureColumn
-   */
-  protected String[] measureColumn;
-  /**
-   * array of carbon measures
-   */
-  protected CarbonMeasure[] carbonMeasures;
-  /**
-   * msrMapping
-   */
-  protected int[] msrMapping;
-  /**
-   * dims
-   */
-  protected boolean[] dimPresent;
-  protected int normLength;
-  /**
-   * actualDimArrays
-   */
-  protected String[] actualDimArray;
-  /**
-   * mrsAggregators
-   */
-  protected String[] msrAggregators;
-  /**
-   * columnAndTableName_ColumnMapForAgg
-   */
-  protected Map<String, String> columnAndTableNameColumnMapForAggMap;
-  /**
-   * denormColumnList
-   */
-  protected List<String> denormColumnList;
-  /**
-   * Member Mapping
-   */
-  protected int[] memberMapping;
-  protected Map<String, String> foreignKeyPrimaryKeyMap;
-  /**
-   * carbondim
-   */
-  private String carbondim;
-  /**
-   * carbonProps
-   */
-  private String carbonProps;
-  /**
-   * carbonmsr
-   */
-  private String carbonmsr;
-  /**
-   * carbonhier
-   */
-  private String carbonhier;
-  /**
-   * carbonMeasureNames
-   */
-  private String carbonMeasureNames;
-  /**
-   * carbonhierColumn
-   */
-  private String carbonhierColumn;
-  /**
-   * carbonTime
-   */
-  private String carbonTime;
-  private String noDictionaryDims;
-  /**
-   * carbonSchema
-   */
-  private String carbonSchema;
-  /**
-   * batchSize
-   */
-  private int batchSize = 10000;
-  /**
-   * isAggregate
-   */
-  private boolean isAggregate;
-  /**
-   * generateDimFiles
-   */
-  private boolean generateDimFiles;
-  /**
-   * storeType
-   */
-  private String storeType;
-  /**
-   * metaHeirSQLQuery
-   */
-  private String metaHeirSQLQuery;
-  /**
-   * Foreign key and respective hierarchy String
-   */
-  private String foreignKeyHierarchyString;
-  /**
-   * heirNadDimsLensString
-   */
-  private String heirNadDimsLensString;
-  /**
-   * measureDataType
-   */
-  private String measureDataType;
-  /**
-   * measureSurrogateRequired
-   */
-  private Map<String, Boolean> measureSurrogateRequired;
-  private String heirKeySize;
-  /**
-   * checkPointFileExits
-   */
-  private String complexDelimiterLevel1;
-  private String complexDelimiterLevel2;
-  private String complexTypeString;
-
-  private String columnPropertiesString;
-
-  private String[] complexTypeColumns;
-  /**
-   * Primary Key String
-   */
-  private String primaryKeysString;
-  /**
-   * foreign key Column name string
-   */
-  private String forgienKeyPrimayKeyString;
-  /**
-   * Primary Key Map.
-   */
-  private Map<String, Boolean> primaryKeyMap;
-  /**
-   *
-   */
-  private Map<String, String> hierDimTableMap;
-  /**
-   * propColumns
-   */
-  private List<String>[] propColumns;
-  /**
-   * propTypes
-   */
-  private List<String>[] propTypes;
-  /**
-   * propIndxs
-   */
-  private int[][] propIndxs;
-  /**
-   * metahierVoList
-   */
-  private List<HierarchiesInfo> metahierVoList;
-  /**
-   * dimesionTableNames
-   */
-  private String dimesionTableNames;
-  /**
-   * column Ids of dimensions in a table
-   */
-  private String dimensionColumnIds;
-  /**
-   * dimTableArray
-   */
-  private String[] dimTableArray;
-  /**
-   * tableName
-   */
-  private String tableName;
-  /**
-   * MOdified Dimension
-   */
-  private String[] modifiedDimension;
-  /**
-   * actualDimNames
-   */
-  private String actualDimNames;
-  private String normHiers;
-  /**
-   * msrAggregatorString
-   */
-  private String msrAggregatorString;
-  /**
-   * columnAndTableName_ColumnMapForAggString
-   */
-  private String columnAndTableNameColumnMapForAggString;
-  private String connectionURL;
-  private String driverClass;
-  private String userName;
-  private String password;
-  /**
-   * denormColumNames
-   */
-  private String denormColumNames;
-
-  /**
-   * databaseName
-   */
-  private String databaseName;
-  /**
-   * partitionID
-   */
-  private String partitionID;
-
-  /**
-   * Id of the load folder
-   */
-  private String segmentId;
-
-  /***
-   * String of columns ordinal and column datatype separated by HASH_SPC_CHARACTER
-   */
-  private String columnSchemaDetails;
-
-  /**
-   * String of key value pair separated by , and HASH_SPC_CHARACTER
-   */
-  private String tableOption;
-
-  /**
-   * wrapper object having the columnSchemaDetails
-   */
-  private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper;
-
-  /**
-   * Wrapper object holding the table options
-   */
-  private TableOptionWrapper tableOptionWrapper;
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-  /**
-   * column data type string.
-   */
-  private String columnsDataTypeString;
-
-  public CarbonCSVBasedSeqGenMeta() {
-    super();
-  }
-
-  public Map<String, GenericDataType> getComplexTypes() {
-    return complexTypes;
-  }
-
-  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 void setComplexTypeString(String complexTypeString) {
-    this.complexTypeString = complexTypeString;
-  }
-
-  public void setColumnPropertiesString(String columnPropertiesString) {
-    this.columnPropertiesString = columnPropertiesString;
-  }
-
-  public String[] getComplexTypeColumns() {
-    return complexTypeColumns;
-  }
-
-  public void setMetaMetaHeirSQLQueries(String metaHeirSQLQuery) {
-    this.metaHeirSQLQuery = metaHeirSQLQuery;
-  }
-
-  public boolean isAggregate() {
-    return isAggregate;
-  }
-
-  public void setAggregate(boolean isAggregate) {
-    this.isAggregate = isAggregate;
-  }
-
-  public int getBatchSize() {
-    return batchSize;
-  }
-
-  public void setBatchSize(int batchSize) {
-    this.batchSize = batchSize;
-  }
-
-  public String getStoreType() {
-    return storeType;
-  }
-
-  public List<HierarchiesInfo> getMetahierVoList() {
-    return metahierVoList;
-  }
-
-  public String getCarbonTime() {
-    return carbonTime;
-  }
-
-  // getters and setters for the step settings
-
-  public void setCarbonProps(String carbonProps) {
-    this.carbonProps = carbonProps;
-  }
-
-  public void setCarbonmsr(String carbonmsr) {
-    this.carbonmsr = carbonmsr;
-  }
-
-  public void setCarbondim(String carbondim) {
-    this.carbondim = carbondim;
-  }
-
-  public void setCarbonhier(String carbonhier) {
-    this.carbonhier = carbonhier;
-  }
-
-  /**
-   * @param connectionURL the connectionURL to set
-   */
-  public void setConnectionURL(String connectionURL) {
-    this.connectionURL = connectionURL;
-  }
-
-  //TODO SIMIAN
-
-  /**
-   * @param driverClass the driverClass to set
-   */
-  public void setDriverClass(String driverClass) {
-    this.driverClass = driverClass;
-  }
-
-  /**
-   * @param userName the userName to set
-   */
-  public void setUserName(String userName) {
-    this.userName = userName;
-  }
-
-  /**
-   * @param password the password to set
-   */
-  public void setPassword(String password) {
-    this.password = password;
-  }
-
-  public void setDateFormat(String dateFormat) { this.dateFormat = dateFormat; }
-
-  /**
-   * set sensible defaults for a new step
-   *
-   * @see StepMetaInterface#setDefault()
-   */
-  public void setDefault() {
-    carbonProps = "";
-    carbondim = "";
-    carbonmsr = "";
-    carbonhier = "";
-    carbonTime = "";
-    driverClass = "";
-    connectionURL = "";
-    userName = "";
-    password = "";
-    carbonSchema = "";
-    storeType = "";
-    isAggregate = false;
-    metaHeirSQLQuery = "";
-    dimesionTableNames = "";
-    dimensionColumnIds = "";
-    noDictionaryDims = "";
-    tableName = "";
-    carbonhierColumn = "";
-    foreignKeyHierarchyString = "";
-    complexTypeString = "";
-    columnPropertiesString = "";
-    complexDelimiterLevel1 = "";
-    complexDelimiterLevel2 = "";
-    primaryKeysString = "";
-    carbonMeasureNames = "";
-    actualDimNames = "";
-    normHiers = "";
-    msrAggregatorString = "";
-    heirKeySize = "";
-    heirNadDimsLensString = "";
-    measureDataType = "";
-    columnAndTableNameColumnMapForAggString = "";
-    denormColumNames = "";
-    partitionID = "";
-    segmentId = "";
-    taskNo = "";
-    columnSchemaDetails = "";
-    columnsDataTypeString = "";
-    tableOption = "";
-    dateFormat = "";
-  }
-
-  // helper method to allocate the arrays
-  public void allocate(int nrkeys) {
-
-  }
-
-  public String getXML() throws KettleValueException {
-    StringBuilder retval = new StringBuilder(150);
-    retval.append("    ").append(XMLHandler.addTagValue("carbonProps", carbonProps));
-    retval.append("    ").append(XMLHandler.addTagValue("dim", carbondim));
-    retval.append("    ").append(XMLHandler.addTagValue("msr", carbonmsr));
-    retval.append("    ").append(XMLHandler.addTagValue("hier", carbonhier));
-    retval.append("    ").append(XMLHandler.addTagValue("time", carbonTime));
-    retval.append("    ").append(XMLHandler.addTagValue("driverClass", driverClass));
-    retval.append("    ").append(XMLHandler.addTagValue("connectionURL", connectionURL));
-    retval.append("    ").append(XMLHandler.addTagValue("userName", userName));
-    retval.append("    ").append(XMLHandler.addTagValue("password", password));
-    retval.append("    ").append(XMLHandler.addTagValue("batchSize", batchSize));
-    retval.append("    ").append(XMLHandler.addTagValue("genDimFiles", generateDimFiles));
-    retval.append("    ").append(XMLHandler.addTagValue("isAggregate", isAggregate));
-    retval.append("    ").append(XMLHandler.addTagValue("storeType", storeType));
-    retval.append("    ").append(XMLHandler.addTagValue("metadataFilePath", metaHeirSQLQuery));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("foreignKeyHierarchyString", foreignKeyHierarchyString));
-    retval.append("    ").append(XMLHandler.addTagValue("complexTypeString", complexTypeString));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("columnPropertiesString", columnPropertiesString));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("complexDelimiterLevel1", complexDelimiterLevel1));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("complexDelimiterLevel2", complexDelimiterLevel2));
-    retval.append("    ").append(XMLHandler.addTagValue("dateFormat", dateFormat));
-    retval.append("    ").append(XMLHandler.addTagValue("primaryKeysString", primaryKeysString));
-    retval.append("    ").append(XMLHandler.addTagValue("carbonMeasureNames", carbonMeasureNames));
-    retval.append("    ").append(XMLHandler.addTagValue("actualDimNames", actualDimNames));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("msrAggregatorString", msrAggregatorString));
-
-    retval.append("    ").append(XMLHandler.addTagValue("dimHierReleation", dimesionTableNames));
-    retval.append("    ").append(XMLHandler.addTagValue("dimensionColumnIds", dimensionColumnIds));
-    retval.append("    ").append(XMLHandler.addTagValue("dimNoDictionary", noDictionaryDims));
-    retval.append("    ").append(XMLHandler.addTagValue("dimColDataTypes", columnsDataTypeString));
-    retval.append("    ").append(XMLHandler.addTagValue("factOrAggTable", tableName));
-    retval.append("    ").append(XMLHandler.addTagValue("carbonhierColumn", carbonhierColumn));
-    retval.append("    ").append(XMLHandler.addTagValue("normHiers", normHiers));
-    retval.append("    ").append(XMLHandler.addTagValue("heirKeySize", heirKeySize));
-
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("forgienKeyPrimayKeyString", forgienKeyPrimayKeyString));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("heirNadDimsLensString", heirNadDimsLensString));
-    retval.append("    ").append(XMLHandler.addTagValue("measureDataType", measureDataType));
-    retval.append("    ").append(XMLHandler.addTagValue("columnAndTableName_ColumnMapForAggString",
-        columnAndTableNameColumnMapForAggString));
-    retval.append("    ").append(XMLHandler.addTagValue("databaseName", databaseName));
-    retval.append("    ").append(XMLHandler.addTagValue("tableName", tableName));
-    retval.append("    ").append(XMLHandler.addTagValue("denormColumNames", denormColumNames));
-    retval.append("    ").append(XMLHandler.addTagValue("partitionID", partitionID));
-    retval.append("    ").append(XMLHandler.addTagValue("segmentId", segmentId));
-    retval.append("    ").append(XMLHandler.addTagValue("taskNo", taskNo));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("columnSchemaDetails", columnSchemaDetails));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("tableOption", tableOption));
-    return retval.toString();
-  }
-
-  public void loadXML(Node stepnode, List<DatabaseMeta> databases, Map<String, Counter> counters)
-      throws KettleXMLException {
-
-    try {
-
-      carbonProps = XMLHandler.getTagValue(stepnode, "carbonProps");
-      carbondim = XMLHandler.getTagValue(stepnode, "dim");
-      carbonmsr = XMLHandler.getTagValue(stepnode, "msr");
-      carbonhier = XMLHandler.getTagValue(stepnode, "hier");
-      carbonTime = XMLHandler.getTagValue(stepnode, "time");
-      driverClass = XMLHandler.getTagValue(stepnode, "driverClass");
-      connectionURL = XMLHandler.getTagValue(stepnode, "connectionURL");
-      userName = XMLHandler.getTagValue(stepnode, "userName");
-      password = XMLHandler.getTagValue(stepnode, "password");
-      carbonhierColumn = XMLHandler.getTagValue(stepnode, "carbonhierColumn");
-      foreignKeyHierarchyString = XMLHandler.getTagValue(stepnode, "foreignKeyHierarchyString");
-      complexTypeString = XMLHandler.getTagValue(stepnode, "complexTypeString");
-      columnPropertiesString = XMLHandler.getTagValue(stepnode, "columnPropertiesString");
-      complexDelimiterLevel1 = XMLHandler.getTagValue(stepnode, "complexDelimiterLevel1");
-      complexDelimiterLevel2 = XMLHandler.getTagValue(stepnode, "complexDelimiterLevel2");
-      dateFormat = XMLHandler.getTagValue(stepnode, "dateFormat");
-      primaryKeysString = XMLHandler.getTagValue(stepnode, "primaryKeysString");
-      carbonMeasureNames = XMLHandler.getTagValue(stepnode, "carbonMeasureNames");
-      actualDimNames = XMLHandler.getTagValue(stepnode, "actualDimNames");
-      normHiers = XMLHandler.getTagValue(stepnode, "normHiers");
-      msrAggregatorString = XMLHandler.getTagValue(stepnode, "msrAggregatorString");
-      heirKeySize = XMLHandler.getTagValue(stepnode, "heirKeySize");
-      forgienKeyPrimayKeyString = XMLHandler.getTagValue(stepnode, "forgienKeyPrimayKeyString");
-      heirNadDimsLensString = XMLHandler.getTagValue(stepnode, "heirNadDimsLensString");
-      measureDataType = XMLHandler.getTagValue(stepnode, "measureDataType");
-      columnAndTableNameColumnMapForAggString =
-          XMLHandler.getTagValue(stepnode, "columnAndTableName_ColumnMapForAggString");
-      dimesionTableNames = XMLHandler.getTagValue(stepnode, "dimHierReleation");
-      dimensionColumnIds = XMLHandler.getTagValue(stepnode, "dimensionColumnIds");
-      noDictionaryDims = XMLHandler.getTagValue(stepnode, "dimNoDictionary");
-      columnsDataTypeString = XMLHandler.getTagValue(stepnode, "dimColDataTypes");
-      tableName = XMLHandler.getTagValue(stepnode, "factOrAggTable");
-      tableName = XMLHandler.getTagValue(stepnode, "tableName");
-      databaseName = XMLHandler.getTagValue(stepnode, "databaseName");
-      denormColumNames = XMLHandler.getTagValue(stepnode, "denormColumNames");
-      partitionID = XMLHandler.getTagValue(stepnode, "partitionID");
-      segmentId = XMLHandler.getTagValue(stepnode, "segmentId");
-      taskNo = XMLHandler.getTagValue(stepnode, "taskNo");
-      columnSchemaDetails = XMLHandler.getTagValue(stepnode, "columnSchemaDetails");
-      tableOption = XMLHandler.getTagValue(stepnode, "tableOption");
-      String batchConfig = XMLHandler.getTagValue(stepnode, "batchSize");
-
-      if (batchConfig != null) {
-        batchSize = Integer.parseInt(batchConfig);
-      }
-
-      String dimeFileConfig = XMLHandler.getTagValue(stepnode, "genDimFiles");
-      if (dimeFileConfig != null) {
-        generateDimFiles = Boolean.parseBoolean(dimeFileConfig);
-      }
-
-      storeType = XMLHandler.getTagValue(stepnode, "storeType");
-      metaHeirSQLQuery = XMLHandler.getTagValue(stepnode, "metadataFilePath");
-
-      isAggregate = "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, "isAggregate"));
-
-      int nrKeys = XMLHandler.countNodes(stepnode, "lookup");
-      allocate(nrKeys);
-
-    } catch (Exception e) {
-      throw new KettleXMLException("Template Plugin Unable to read step info from XML node", e);
-    }
-
-  }
-
-  public void initialize() throws KettleException {
-    this.columnSchemaDetailsWrapper = new ColumnSchemaDetailsWrapper(columnSchemaDetails);
-    this.tableOptionWrapper = TableOptionWrapper.getTableOptionWrapperInstance();
-    tableOptionWrapper.populateTableOptions(tableOption);
-
-    updateDimensions(carbondim, carbonmsr, noDictionaryDims);
-    dimColDataTypes = NonDictionaryUtil.extractDimColsDataTypeValues(columnsDataTypeString);
-    if (null != complexTypeString) {
-      complexTypes = getComplexTypesMap(complexTypeString);
-    } else {
-      complexTypeColumns = new String[0];
-    }
-
-    if (null != columnPropertiesString) {
-      updateColumnPropertiesMap(columnPropertiesString);
-    }
-    hirches = getHierarichies(carbonhier);
-
-    hierColumnMap = getHierarchiesColumnMap(carbonhierColumn);
-
-    foreignKeyHierarchyMap = getForeignKeyHierMap(foreignKeyHierarchyString);
-
-    primaryKeyMap = updatePrimaryKeyMap(primaryKeysString);
-
-    foreignKeyPrimaryKeyMap = getForeignKeyColumnNameMap(forgienKeyPrimayKeyString);
-
-    actualDimArray = getActualDimensionArray(actualDimNames);
-
-    //update non time dimension properties
-    updateDimProperties();
-
-    updateMeasureAggregator(msrAggregatorString);
-
-    measureSurrogateRequired = getMeasureDatatypeMap(measureDataType);
-
-    updateHierDimTableMap(dimesionTableNames);
-
-    if (isAggregate) {
-      columnAndTableNameColumnMapForAggMap =
-          new HashMap<String, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-      String[] split =
-          columnAndTableNameColumnMapForAggString.split(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      for (int i = 0; i < split.length; i++) {
-        String[] split2 = split[i].split(CarbonCommonConstants.HYPHEN_SPC_CHARACTER);
-        columnAndTableNameColumnMapForAggMap.put(split2[0], split2[1]);
-      }
-    }
-
-    updateDenormColunList(denormColumNames);
-  }
-
-  private void updateColumnPropertiesMap(String columnPropertiesString) {
-    String[] colsProperty = columnPropertiesString.split(CarbonCommonConstants.HASH_SPC_CHARACTER);
-    for (String property : colsProperty) {
-      String[] colKeyVals = property.split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      String colName = colKeyVals[0];
-      Map<String, String> colPropMap = new HashMap<>();
-      String[] keyVals = colKeyVals[1].split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-      for (int i = 0; i < keyVals.length; i++) {
-        String[] keyVal = keyVals[i].split(CarbonCommonConstants.HYPHEN_SPC_CHARACTER);
-        String key = keyVal[0];
-        String value = keyVal[1];
-        colPropMap.put(key, value);
-      }
-      columnProperties.put(colName, colPropMap);
-    }
-  }
-
-  private void updateDenormColunList(String denormColumNames) {
-    //
-    if (null == denormColumNames || "".equals(denormColumNames)) {
-      denormColumnList = new ArrayList<String>(1);
-      return;
-    }
-
-    String[] columnNames = denormColumNames.split(CarbonCommonConstants.HASH_SPC_CHARACTER);
-
-    if (null == denormColumnList) {
-      denormColumnList = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    }
-
-    for (String colName : columnNames) {
-      denormColumnList.add(colName);
-    }
-  }
-
-  private void updateHierDimTableMap(String dimesionTableNames) {
-    if (null == dimesionTableNames || "".equals(dimesionTableNames)) {
-      return;
-    }
-
-    String[] hierTableName =
-        dimesionTableNames.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-    if (null == hierDimTableMap) {
-      hierDimTableMap = new HashMap<String, String>(hierTableName.length);
-    }
-    if (null == dimTableArray) {
-      dimTableArray = new String[hierTableName.length];
-    }
-    int i = 0;
-    for (String hierTable : hierTableName) {
-      String[] hierAndTable = hierTable.split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      hierDimTableMap.put(hierAndTable[0], hierAndTable[1]);
-      dimTableArray[i++] = hierAndTable[1];
-    }
-  }
-
-  private Map<String, Boolean> getMeasureDatatypeMap(String measureDataType) {
-    return new HashMap<String, Boolean>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  private void updateMeasureAggregator(String msrAggregatorString) {
-    String[] split = null;
-    if (msrAggregatorString == null) {
-      split =  new String[0];
-    } else {
-      split = msrAggregatorString.split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER);
-    }
-    msrAggregators = new String[split.length];
-    System.arraycopy(split, 0, msrAggregators, 0, split.length);
-  }
-
-  private String[] getActualDimensionArray(String actualDimNames) {
-    if (actualDimNames == null || "".equals(actualDimNames)) {
-      return new String[0];
-    }
-
-    return actualDimNames.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-  }
-
-  private List<String> getNormHierList(String normHier) {
-    List<String> hierList = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    if (null != normHier && normHier.length() != 0) {
-      String[] split = normHier.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-
-      for (int i = 0; i < split.length; i++) {
-        hierList.add(split[i]);
-      }
-    }
-    return hierList;
-  }
-
-  private Map<String, String> getForeignKeyColumnNameMap(String foreignKeyColumnNameString) {
-    if (foreignKeyColumnNameString == null || "".equals(foreignKeyColumnNameString)) {
-      return new HashMap<String, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    }
-    Map<String, String> map =
-        new HashMap<String, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    String[] foreignKeys =
-        foreignKeyColumnNameString.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-    for (int i = 0; i < foreignKeys.length; i++) {
-      String[] foreignHierArray = foreignKeys[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      String hiers = map.get(foreignHierArray[0]);
-
-      if (null == hiers) {
-        map.put(foreignHierArray[0], foreignHierArray[1]);
-      } else {
-        map.put(foreignHierArray[0],
-            hiers + CarbonCommonConstants.COMA_SPC_CHARACTER + foreignHierArray[1]);
-      }
-    }
-    return map;
-  }
-
-  private Map<String, Boolean> updatePrimaryKeyMap(String primaryKeysString) {
-    if (primaryKeysString == null || "".equals(primaryKeysString)) {
-      return new HashMap<String, Boolean>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    }
-    Map<String, Boolean> resultMap =
-        new HashMap<String, Boolean>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    String[] primaryKeys = primaryKeysString.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-    for (int i = 0; i < primaryKeys.length; i++) {
-      resultMap.put(primaryKeys[i], true);
-    }
-    return resultMap;
-  }
-
-  public void updateHierMappings(RowMetaInterface metaInterface) {
-    List<String> actualHierList = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    for (int j = 0; j < metaInterface.size(); j++) {
-      String foreignKey = metaInterface.getValueMeta(j).getName();
-      String actualHier = foreignKeyHierarchyMap.get(foreignKey);
-      if (null != actualHier) {
-        if (actualHier.contains(CarbonCommonConstants.COMA_SPC_CHARACTER)) {
-          String[] splitHier = actualHier.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-          for (String hier : splitHier) {
-            actualHierList.add(hier);
-          }
-        } else {
-          actualHierList.add(actualHier);
-        }
-      }
-    }
-
-    hierNames = new String[actualHierList.size()];
-    hierNames = actualHierList.toArray(new String[actualHierList.size()]);
-  }
-
-  private Map<String, GenericDataType> getComplexTypesMap(String complexTypeString) {
-    Map<String, GenericDataType> complexTypesMap = new LinkedHashMap<String, GenericDataType>();
-    String[] hierarchies = complexTypeString.split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER);
-    complexTypeColumns = new String[hierarchies.length];
-    for (int i = 0; i < hierarchies.length; i++) {
-      String[] levels = hierarchies[i].split(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      String[] levelInfo = levels[0].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      GenericDataType g = levelInfo[1].equals(CarbonCommonConstants.ARRAY) ?
-          new ArrayDataType(levelInfo[0], "", levelInfo[3]) :
-          new StructDataType(levelInfo[0], "", levelInfo[3]);
-      complexTypesMap.put(levelInfo[0], g);
-      complexTypeColumns[i] = levelInfo[0];
-      for (int j = 1; j < levels.length; j++) {
-        levelInfo = levels[j].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-        switch (levelInfo[1]) {
-          case CarbonCommonConstants.ARRAY:
-            g.addChildren(new ArrayDataType(levelInfo[0], levelInfo[2], levelInfo[3]));
-            break;
-          case CarbonCommonConstants.STRUCT:
-            g.addChildren(new StructDataType(levelInfo[0], levelInfo[2], levelInfo[3]));
-            break;
-          default:
-            g.addChildren(new PrimitiveDataType(levelInfo[0], levelInfo[2], levelInfo[3],
-                Integer.parseInt(levelInfo[4])));
-        }
-      }
-    }
-    return complexTypesMap;
-  }
-
-  private Map<String, String> getForeignKeyHierMap(String foreignKeyHierarchyString) {
-    if (foreignKeyHierarchyString == null || "".equals(foreignKeyHierarchyString)) {
-      return new HashMap<String, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    }
-    Map<String, String> map =
-        new HashMap<String, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    String[] hies = foreignKeyHierarchyString.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-    for (int i = 0; i < hies.length; i++) {
-      String[] foreignHierArray = hies[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      String hiers = map.get(foreignHierArray[0]);
-
-      if (null == hiers) {
-        map.put(foreignHierArray[0], foreignHierArray[1]);
-      } else {
-        map.put(foreignHierArray[0],
-            hiers + CarbonCommonConstants.COMA_SPC_CHARACTER + foreignHierArray[1]);
-      }
-
-    }
-    return map;
-  }
-
-  private Map<String, String[]> getHierarchiesColumnMap(String carbonhierColumn) {
-    if (carbonhierColumn == null || "".equals(carbonhierColumn)) {
-      return new HashMap<String, String[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    }
-    Map<String, String[]> map =
-        new HashMap<String, String[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    String[] hies = carbonhierColumn.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-    for (int i = 0; i < hies.length; i++) {
-      String hie = hies[i];
-
-      String hierName = hie.substring(0, hie.indexOf(CarbonCommonConstants.COLON_SPC_CHARACTER));
-
-      String[] columnArray = getStringArray(hie.substring(
-          hie.indexOf(CarbonCommonConstants.COLON_SPC_CHARACTER)
-              + CarbonCommonConstants.COLON_SPC_CHARACTER.length(), hie.length()));
-      map.put(hierName, columnArray);
-    }
-    return map;
-  }
-
-  private String[] getStringArray(String columnNames) {
-    String[] splitedColumnNames = columnNames.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-    String[] columns = new String[splitedColumnNames.length];
-
-    System.arraycopy(splitedColumnNames, 0, columns, 0, columns.length);
-    return columns;
-  }
-
-  /**
-   * Parse the properties string.
-   * Level Entries separated by '&'
-   * Level and prop details separated by ':'
-   * Property column name and index separated by ','
-   * Level:p1,index1:p2,index2&Level2....
-   */
-  private void updateDimProperties() {
-    Map<String, int[]> indices =
-        new HashMap<String, int[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    Map<String, String[]> columns =
-        new HashMap<String, String[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    Map<String, String[]> dbTypes =
-        new HashMap<String, String[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    if (carbonProps != null && !"".equals(carbonProps)) {
-      String[] entries = carbonProps.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-      for (int i = 0; i < entries.length; i++) {
-        String[] levelEntry = entries[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-        String dimColumnName = levelEntry[0];
-        int[] pIndices = new int[levelEntry.length - 1];
-        String[] cols = new String[levelEntry.length - 1];
-        String[] dbType = new String[levelEntry.length - 1];
-        for (int j = 1; j < levelEntry.length; j++) {
-          String[] propEntry = levelEntry[j].split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-          pIndices[j - 1] = Integer.parseInt(propEntry[1]);
-
-          cols[j - 1] = propEntry[0];
-          dbType[j - 1] = propEntry[2];
-        }
-
-        indices.put(dimColumnName, pIndices);
-        columns.put(dimColumnName, cols);
-        dbTypes.put(dimColumnName, dbType);
-      }
-    }
-
-    if (indices.isEmpty()) {
-      return;
-    }
-
-    propColumns = new List[dimColNames.length];
-    propTypes = new List[dimColNames.length];
-    propIndxs = new int[dimColNames.length][];
-
-    //Fill the property details based on the map created
-    for (int i = 0; i < dimColNames.length; i++) {
-      //Properties present or not
-      if (indices.containsKey(dimColNames[i])) {
-        propColumns[i] = Arrays.asList(columns.get(dimColNames[i]));
-        propTypes[i] = Arrays.asList(dbTypes.get(dimColNames[i]));
-        propIndxs[i] = indices.get(dimColNames[i]);
-      } else {
-        propColumns[i] = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-        propTypes[i] = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-        propIndxs[i] = new int[0];
-      }
-    }
-  }
-
-  private Map<String, int[]> getHierarichies(String ds) {
-    if (ds == null || "".equals(ds)) {
-      return new HashMap<String, int[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    }
-    Map<String, int[]> map =
-        new HashMap<String, int[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    String[] hies = ds.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-    for (int i = 0; i < hies.length; i++) {
-      String hie = hies[i];
-
-      String name = hie.substring(0, hie.indexOf(CarbonCommonConstants.COLON_SPC_CHARACTER));
-
-      int[] a = getIntArray(hie.substring(hie.indexOf(CarbonCommonConstants.COLON_SPC_CHARACTER)
-          + CarbonCommonConstants.COLON_SPC_CHARACTER.length(), hie.length()));
-      map.put(name, a);
-    }
-    return map;
-  }
-
-  private int[] getIntArray(String ds) {
-
-    String[] sp = ds.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-    int[] a = new int[sp.length];
-
-    for (int i = 0; i < a.length; i++) {
-      a[i] = Integer.parseInt(sp[i]);
-    }
-    return a;
-
-  }
-
-  private void updateDimensions(String ds, String msr, String noDictionaryDims) {
-    String[] sp = null;
-    if (null != ds) {
-      sp = ds.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-    } else {
-      sp = new String[0];
-    }
-    int[] dimsLocal = new int[sp.length];
-    int[] lens = new int[sp.length];
-    List<String> list = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    dimPresent = new boolean[sp.length];
-
-    for (int i = 0; i < dimsLocal.length; i++) {
-      String[] dim = sp[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      list.add(dim[0]);
-      dimsLocal[i] = Integer.parseInt(dim[1]);
-      lens[i] = Integer.parseInt(dim[2]);
-
-      if ("Y".equals(dim[3])) {
-        dimPresent[i] = true;
-        normLength++;
-      }
-    }
-    dims = dimsLocal;
-    dimLens = lens;
-    dimColNames = list.toArray(new String[list.size()]);
-
-    // get high cardinality dimension Array
-    noDictionaryCols = NonDictionaryUtil.extractNoDictionaryDimsArr(noDictionaryDims);
-
-    String[] sm = null;
-    if (null != msr) {
-      sm = msr.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-    } else {
-      sm = new String[0];
-    }
-    int[] m = new int[sm.length];
-    Set<String> mlist = new LinkedHashSet<String>();
-    for (int i = 0; i < m.length; i++) {
-      String[] ms = sm[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      mlist.add(ms[0]);
-      m[i] = Integer.parseInt(ms[1]);
-    }
-    msrs = m;
-    measureColumn = mlist.toArray(new String[mlist.size()]);
-  }
-
-  public void readRep(Repository rep, ObjectId idStep, List<DatabaseMeta> databases,
-      Map<String, Counter> counters) throws KettleException {
-    try {
-      //
-      carbonProps = rep.getStepAttributeString(idStep, "carbonProps");
-      carbonmsr = rep.getStepAttributeString(idStep, "msr");
-      carbondim = rep.getStepAttributeString(idStep, "dim");
-      carbonhier = rep.getStepAttributeString(idStep, "hier");
-      carbonTime = rep.getStepAttributeString(idStep, "time");
-      //
-      driverClass = rep.getStepAttributeString(idStep, "driverClass");
-      connectionURL = rep.getStepAttributeString(idStep, "connectionURL");
-      userName = rep.getStepAttributeString(idStep, "userName");
-      password = rep.getStepAttributeString(idStep, "password");
-      isAggregate = rep.getStepAttributeBoolean(idStep, "isAggregate");
-      metaHeirSQLQuery = rep.getStepAttributeString(idStep, "metadataFilePath");
-      carbonhierColumn = rep.getStepAttributeString(idStep, "carbonhierColumn");
-      foreignKeyHierarchyString = rep.getStepAttributeString(idStep, "foreignKeyHierarchyString");
-      primaryKeysString = rep.getStepAttributeString(idStep, "primaryKeysString");
-      carbonMeasureNames = rep.getStepAttributeString(idStep, "carbonMeasureNames");
-      actualDimNames = rep.getStepAttributeString(idStep, "actualDimNames");
-      msrAggregatorString = rep.getStepAttributeString(idStep, "msrAggregatorString");
-
-      dimesionTableNames = rep.getStepAttributeString(idStep, "dimHierReleation");
-      dimensionColumnIds = rep.getStepAttributeString(idStep, "dimensionColumnIds");
-      noDictionaryDims = rep.getStepAttributeString(idStep, "dimNoDictionary");
-      columnsDataTypeString = rep.getStepAttributeString(idStep, "dimColDataTypes");
-      normHiers = rep.getStepAttributeString(idStep, "normHiers");
-      tableName = rep.getStepAttributeString(idStep, "factOrAggTable");
-      batchSize = Integer.parseInt(rep.getStepAttributeString(idStep, "batchSize"));
-      heirKeySize = rep.getStepAttributeString(idStep, "heirKeySize");
-      forgienKeyPrimayKeyString = rep.getStepAttributeString(idStep, "forgienKeyPrimayKeyString");
-      heirNadDimsLensString = rep.getStepAttributeString(idStep, "heirNadDimsLensString");
-      measureDataType = rep.getStepAttributeString(idStep, "measureDataType");
-      columnAndTableNameColumnMapForAggString =
-          rep.getStepAttributeString(idStep, "columnAndTableName_ColumnMapForAggString");
-      databaseName = rep.getStepAttributeString(idStep, "databaseName");
-
-      tableName = rep.getStepAttributeString(idStep, "tableName");
-      denormColumNames = rep.getStepAttributeString(idStep, "denormColumNames");
-      partitionID = rep.getStepAttributeString(idStep, "partitionID");
-      segmentId = rep.getStepAttributeString(idStep, "segmentId");
-      taskNo = rep.getStepAttributeString(idStep, "taskNo");
-      columnSchemaDetails = rep.getStepAttributeString(idStep, "columnSchemaDetails");
-      tableOption = rep.getStepAttributeString(idStep, "tableOption");
-      dateFormat = rep.getStepAttributeString(idStep, "dateFormat");
-      int nrKeys = rep.countNrStepAttributes(idStep, "lookup_keyfield");
-      allocate(nrKeys);
-    } catch (Exception e) {
-      throw new KettleException(
-          BaseMessages.getString(pkg, "CarbonStep.Exception.UnexpectedErrorInReadingStepInfo"), e);
-    }
-  }
-
-  public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep)
-      throws KettleException {
-    try {
-      rep.saveStepAttribute(idTransformation, idStep, "dim", carbondim);
-      rep.saveStepAttribute(idTransformation, idStep, "carbonProps", carbonProps);
-      rep.saveStepAttribute(idTransformation, idStep, "msr", carbonmsr);
-      rep.saveStepAttribute(idTransformation, idStep, "hier", carbonhier);
-      rep.saveStepAttribute(idTransformation, idStep, "carbonhierColumn", carbonhierColumn);
-      rep.saveStepAttribute(idTransformation, idStep, "columnAndTableName_ColumnMapForAggString",
-          columnAndTableNameColumnMapForAggString);
-      rep.saveStepAttribute(idTransformation, idStep, "time", carbonTime);
-      rep.saveStepAttribute(idTransformation, idStep, "driverClass", driverClass);
-      rep.saveStepAttribute(idTransformation, idStep, "connectionURL", connectionURL);
-      rep.saveStepAttribute(idTransformation, idStep, "userName", userName);
-      rep.saveStepAttribute(idTransformation, idStep, "password", password);
-      rep.saveStepAttribute(idTransformation, idStep, "isInitialLoad", isAggregate);
-      rep.saveStepAttribute(idTransformation, idStep, "metadataFilePath", metaHeirSQLQuery);
-      rep.saveStepAttribute(idTransformation, idStep, "batchSize", batchSize);
-      rep.saveStepAttribute(idTransformation, idStep, "dimHierReleation", dimesionTableNames);
-      rep.saveStepAttribute(idTransformation, idStep, "dimensionColumnIds", dimensionColumnIds);
-      rep.saveStepAttribute(idTransformation, idStep, "dimNoDictionary", noDictionaryDims);
-      rep.saveStepAttribute(idTransformation, idStep, "dimColDataTypes", columnsDataTypeString);
-      rep.saveStepAttribute(idTransformation, idStep, "foreignKeyHierarchyString",
-          foreignKeyHierarchyString);
-      rep.saveStepAttribute(idTransformation, idStep, "primaryKeysString", primaryKeysString);
-      rep.saveStepAttribute(idTransformation, idStep, "carbonMeasureNames", carbonMeasureNames);
-      rep.saveStepAttribute(idTransformation, idStep, "actualDimNames", actualDimNames);
-      rep.saveStepAttribute(idTransformation, idStep, "normHiers", normHiers);
-      rep.saveStepAttribute(idTransformation, idStep, "msrAggregatorString", msrAggregatorString);
-      rep.saveStepAttribute(idTransformation, idStep, "heirKeySize", heirKeySize);
-      rep.saveStepAttribute(idTransformation, idStep, "forgienKeyPrimayKeyString",
-          forgienKeyPrimayKeyString);
-      rep.saveStepAttribute(idTransformation, idStep, "factOrAggTable", tableName);
-      rep.saveStepAttribute(idTransformation, idStep, "heirNadDimsLensString",
-          heirNadDimsLensString);
-      rep.saveStepAttribute(idTransformation, idStep, "measureDataType", measureDataType);
-      rep.saveStepAttribute(idTransformation, idStep, "databaseName", databaseName);
-      rep.saveStepAttribute(idTransformation, idStep, "tableName", tableName);
-      rep.saveStepAttribute(idTransformation, idStep, "denormColumNames", denormColumNames);
-      rep.saveStepAttribute(idTransformation, idStep, "partitionID", partitionID);
-      rep.saveStepAttribute(idTransformation, idStep, "segmentId", segmentId);
-      rep.saveStepAttribute(idTransformation, idStep, "taskNo", taskNo);
-      rep.saveStepAttribute(idTransformation, idStep, "columnSchemaDetails", columnSchemaDetails);
-      rep.saveStepAttribute(idTransformation, idStep, "tableOption", tableOption);
-      rep.saveStepAttribute(idTransformation, idStep, "dateFormat", dateFormat);
-    } catch (Exception e) {
-      throw new KettleException(
-          BaseMessages.getString(pkg, "CarbonStep.Exception.UnableToSaveStepInfoToRepository")
-              + idStep, e);
-    }
-  }
-
-  public void check(List<CheckResultInterface> remarks, TransMeta transmeta, StepMeta stepMeta,
-      RowMetaInterface prev, String[] input, String[] output, RowMetaInterface info) {
-    CarbonDataProcessorUtil.check(pkg, remarks, stepMeta, prev, input);
-  }
-
-  public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int cnr,
-      TransMeta transMeta, Trans disp) {
-    return new CarbonCSVBasedSeqGenStep(stepMeta, stepDataInterface, cnr, transMeta, disp);
-  }
-
-  public StepDataInterface getStepData() {
-    return new CarbonCSVBasedSeqGenData();
-  }
-
-  public List<String>[] getPropertiesColumns() {
-    return propColumns;
-  }
-
-  public int[][] getPropertiesIndices() {
-    return propIndxs;
-  }
-
-  public List<String>[] getPropTypes() {
-    return propTypes;
-  }
-
-  public void setTableNames(String dimHierReleation) {
-    this.dimesionTableNames = dimHierReleation;
-  }
-
-  /**
-   * @return column Ids
-   */
-  public String[] getDimensionColumnIds() {
-    return null != dimensionColumnIds ?
-        dimensionColumnIds.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER) :
-        new String[0];
-  }
-
-  /**
-   * @param dimensionColumnIds column Ids for dimensions in a table
-   */
-  public void setDimensionColumnIds(String dimensionColumnIds) {
-    this.dimensionColumnIds = dimensionColumnIds;
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  public void setModifiedDimension(String[] modifiedDimension) {
-    this.modifiedDimension = modifiedDimension;
-  }
-
-  public void setCarbonhierColumn(String carbonhierColumn) {
-    this.carbonhierColumn = carbonhierColumn;
-  }
-
-  public void setForeignKeyHierarchyString(String foreignKeyHierarchyString) {
-    this.foreignKeyHierarchyString = foreignKeyHierarchyString;
-  }
-
-  public void setPrimaryKeysString(String primaryKeysString) {
-    this.primaryKeysString = primaryKeysString;
-  }
-
-  public Map<String, Boolean> getPrimaryKeyMap() {
-    return primaryKeyMap;
-  }
-
-  public void setCarbonMeasureNames(String carbonMeasureNames) {
-    this.carbonMeasureNames = carbonMeasureNames;
-  }
-
-  public void setActualDimNames(String actualDimNames) {
-    this.actualDimNames = actualDimNames;
-  }
-
-  public void setNormHiers(String normHiers) {
-    this.normHiers = normHiers;
-  }
-
-  public void setMsrAggregatorString(String msrAggregatorString) {
-    this.msrAggregatorString = msrAggregatorString;
-  }
-
-  public void setHeirKeySize(String heirKeySize) {
-    this.heirKeySize = heirKeySize;
-  }
-
-  public void setForgienKeyPrimayKeyString(String forgienKeyPrimayKeyString) {
-    this.forgienKeyPrimayKeyString = forgienKeyPrimayKeyString;
-  }
-
-  public void setHeirNadDimsLensString(String heirNadDimsLensString) {
-    this.heirNadDimsLensString = heirNadDimsLensString;
-  }
-
-  public void setMeasureDataType(String measureDataType) {
-    this.measureDataType = measureDataType;
-  }
-
-  public Map<String, Boolean> getMeasureSurrogateRequired() {
-    return measureSurrogateRequired;
-  }
-
-  public Map<String, String> getHierDimTableMap() {
-    return hierDimTableMap;
-  }
-
-  public String[] getDimTableArray() {
-    return dimTableArray;
-  }
-
-  public void setColumnAndTableNameColumnMapForAggString(
-      String columnAndTableNameColumnMapForAggString) {
-    this.columnAndTableNameColumnMapForAggString = columnAndTableNameColumnMapForAggString;
-  }
-
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  public void setDenormColumNames(String denormColumNames) {
-    this.denormColumNames = denormColumNames;
-  }
-
-  public String getNoDictionaryDims() {
-    return noDictionaryDims;
-  }
-
-  public void setNoDictionaryDims(String noDictionaryDims) {
-    this.noDictionaryDims = noDictionaryDims;
-  }
-
-  /**
-   * @param columnsDataTypeString
-   */
-  public void setDimensionColumnsDataType(String columnsDataTypeString) {
-    this.columnsDataTypeString = columnsDataTypeString;
-
-  }
-
-  /**
-   * @return partitionId
-   */
-  public String getPartitionID() {
-    return partitionID;
-  }
-
-  /**
-   * @param partitionID
-   */
-  public void setPartitionID(String partitionID) {
-    this.partitionID = partitionID;
-  }
-
-  /**
-   * set the the serialized String of columnSchemaDetails
-   *
-   * @param columnSchemaDetails
-   */
-  public void setColumnSchemaDetails(String columnSchemaDetails) {
-    this.columnSchemaDetails = columnSchemaDetails;
-  }
-
-  /**
-   * return segmentId
-   *
-   * @return
-   */
-  public String getSegmentId() {
-    return segmentId;
-  }
-
-  /**
-   * set segment Id
-   *
-   * @param segmentId
-   */
-  public void setSegmentId(String segmentId) {
-    this.segmentId = segmentId;
-  }
-
-  /**
-   * @param taskNo
-   */
-  public void setTaskNo(String taskNo) {
-    this.taskNo = taskNo;
-  }
-
-  /**
-   * @return
-   */
-  public String getTaskNo() {
-    return taskNo;
-  }
-
-  public Map<String, Map<String, String>> getColumnPropertiesMap() {
-    return columnProperties;
-  }
-
-  /**
-   * returns wrapper object having the columnSchemaDetails
-   *
-   * @return
-   */
-  public ColumnSchemaDetailsWrapper getColumnSchemaDetailsWrapper() {
-    return columnSchemaDetailsWrapper;
-  }
-
-  /**
-   * the method set the TableOption details
-   * @param tableOption
-   */
-  public void setTableOption(String tableOption) {
-    this.tableOption = tableOption;
-  }
-
-  /**
-   * the method returns the wrapper object of tableoption
-   * @return
-   */
-  public TableOptionWrapper getTableOptionWrapper() {
-    return tableOptionWrapper;
-  }
-}
-


[03/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
deleted file mode 100644
index f037cf0..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
+++ /dev/null
@@ -1,2009 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.charset.Charset;
-import java.sql.Connection;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.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.common.logging.impl.StandardLogService;
-import org.apache.carbondata.core.cache.dictionary.Dictionary;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.keygenerator.factory.KeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-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.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.writer.ByteArrayHolder;
-import org.apache.carbondata.core.writer.HierarchyValueWriterForCSV;
-import org.apache.carbondata.processing.constants.LoggerAction;
-import org.apache.carbondata.processing.dataprocessor.manager.CarbonDataProcessorManager;
-import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.mdkeygen.file.FileData;
-import org.apache.carbondata.processing.mdkeygen.file.FileManager;
-import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
-import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetails;
-import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper;
-import org.apache.carbondata.processing.schema.metadata.ColumnsInfo;
-import org.apache.carbondata.processing.schema.metadata.HierarchiesInfo;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-import org.apache.carbondata.processing.util.NonDictionaryUtil;
-import static org.apache.carbondata.processing.constants.TableOptionConstant.BAD_RECORDS_ACTION;
-import static org.apache.carbondata.processing.constants.TableOptionConstant.BAD_RECORDS_LOGGER_ENABLE;
-import static org.apache.carbondata.processing.constants.TableOptionConstant.SERIALIZATION_NULL_FORMAT;
-
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.core.row.ValueMeta;
-import org.pentaho.di.core.row.ValueMetaInterface;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStep;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-
-public class CarbonCSVBasedSeqGenStep extends BaseStep {
-
-  /**
-   * BYTE ENCODING
-   */
-  public static final String BYTE_ENCODING = "ISO-8859-1";
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonCSVBasedSeqGenStep.class.getName());
-  /**
-   * NUM_CORES_DEFAULT_VAL
-   */
-  private static final int NUM_CORES_DEFAULT_VAL = 2;
-  /**
-   * drivers
-   */
-  private static final Map<String, String> DRIVERS;
-
-  static {
-
-    DRIVERS = new HashMap<String, String>(16);
-    DRIVERS.put("oracle.jdbc.OracleDriver", CarbonCommonConstants.TYPE_ORACLE);
-    DRIVERS.put("com.mysql.jdbc.Driver", CarbonCommonConstants.TYPE_MYSQL);
-    DRIVERS.put("org.gjt.mm.mysql.Driver", CarbonCommonConstants.TYPE_MYSQL);
-    DRIVERS.put("com.microsoft.sqlserver.jdbc.SQLServerDriver", CarbonCommonConstants.TYPE_MSSQL);
-    DRIVERS.put("com.sybase.jdbc3.jdbc.SybDriver", CarbonCommonConstants.TYPE_SYBASE);
-  }
-
-  /**
-   * ReentrantLock getRowLock
-   */
-  private final Object getRowLock = new Object();
-  /**
-   * ReentrantLock putRowLock
-   */
-  private final Object putRowLock = new Object();
-  /**
-   * CarbonSeqGenData
-   */
-  private CarbonCSVBasedSeqGenData data;
-  /**
-   * CarbonSeqGenStepMeta1
-   */
-  private CarbonCSVBasedSeqGenMeta meta;
-  /**
-   * Map of Connection
-   */
-  private Map<String, Connection> cons = new HashMap<>(16);
-  /**
-   * Csv file path
-   */
-  private String csvFilepath;
-
-  /**
-   * badRecordsLogger
-   */
-  private BadRecordsLogger badRecordsLogger;
-  /**
-   * Normalized Hier and HierWriter map
-   */
-  private Map<String, HierarchyValueWriterForCSV> nrmlizedHierWriterMap =
-      new HashMap<String, HierarchyValueWriterForCSV>(16);
-  /**
-   * load Folder location
-   */
-  private String loadFolderLoc;
-  /**
-   * File manager
-   */
-  private IFileManagerComposite filemanager;
-  /**
-   * measureCol
-   */
-  private List<String> measureCol;
-  /**
-   * dimPresentCsvOrder - Dim present In CSV order
-   */
-  private boolean[] dimPresentCsvOrder;
-  /**
-   * propMap
-   */
-  private Map<String, int[]> propMap;
-  /**
-   * resultArray
-   */
-  private Future[] resultArray;
-
-  /**
-   * denormHierarchies
-   */
-  private List<String> denormHierarchies;
-  /**
-   * readCounter
-   */
-  private long readCounter;
-  /**
-   * writeCounter
-   */
-  private long writeCounter;
-  /**
-   * logCounter
-   */
-  private int logCounter;
-  /**
-   * presentColumnMapIndex
-   */
-  private int[] presentColumnMapIndex;
-  /**
-   * measurePresentMapping
-   */
-  private boolean[] measurePresentMapping;
-  /**
-   * measureSurrogateReqMapping
-   */
-  private boolean[] measureSurrogateReqMapping;
-  /**
-   * foreignKeyMappingColumns
-   */
-  private String[] foreignKeyMappingColumns;
-  /**
-   * foreignKeyMappingColumns
-   */
-  private String[][] foreignKeyMappingColumnsForMultiple;
-  /**
-   * Meta column names
-   */
-  private String[] metaColumnNames;
-  /**
-   * duplicateColMapping
-   */
-  private int[][] duplicateColMapping;
-  private ExecutorService exec;
-  /**
-   * threadStatusObserver
-   */
-  private ThreadStatusObserver threadStatusObserver;
-  /**
-   * CarbonCSVBasedDimSurrogateKeyGen
-   */
-  private CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen;
-
-  private DataType[] msrDataType;
-  /**
-   * wrapper object having the columnSchemaDetails
-   */
-  private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper;
-
-  /**
-   * to check whether column is a no dicitonary column or not
-   */
-  private boolean[] isNoDictionaryColumn;
-  /**
-   * to check whether column is a no dicitonary column or not
-   */
-  private boolean[] isStringDataType;
-  /**
-   * to check whether column is a no dicitonary column or not
-   */
-  private String[] dataTypes;
-
-  /**
-   * to check whether column is complex type column or not
-   */
-  private boolean[] isComplexTypeColumn;
-
-  /**
-   * to store index of no dictionapry column
-   */
-  private int[] noDictionaryAndComplexIndexMapping;
-
-  private GenericDataType[] complexTypes;
-
-  private DirectDictionaryGenerator[] directDictionaryGenerators;
-  /**
-   * dimension column ids
-   */
-  private String[] dimensionColumnIds;
-
-  private Trans dis;
-
-  /**
-   * Constructor
-   *
-   * @param s
-   * @param stepDataInterface
-   * @param c
-   * @param t
-   * @param dis
-   */
-  public CarbonCSVBasedSeqGenStep(StepMeta s, StepDataInterface stepDataInterface, int c,
-      TransMeta t, Trans dis) {
-    super(s, stepDataInterface, c, t, dis);
-    csvFilepath = dis.getVariable("csvInputFilePath");
-    this.dis = dis;
-
-  }
-
-  /**
-   * processRow
-   */
-  public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException {
-
-    try {
-      meta = (CarbonCSVBasedSeqGenMeta) smi;
-      StandardLogService.setThreadName(meta.getPartitionID(), null);
-      data = (CarbonCSVBasedSeqGenData) sdi;
-
-      Object[] r = getRow();  // get row, blocks when needed!
-      if (first) {
-        CarbonTimeStatisticsFactory.getLoadStatisticsInstance()
-                .recordGeneratingDictionaryValuesTime(meta.getPartitionID(),
-                        System.currentTimeMillis());
-        first = false;
-        meta.initialize();
-        final Object dataProcessingLockObject = CarbonDataProcessorManager.getInstance()
-            .getDataProcessingLockObject(meta.getDatabaseName() + '_' + meta.getTableName());
-        synchronized (dataProcessingLockObject) {
-          // observer of writing file in thread
-          this.threadStatusObserver = new ThreadStatusObserver();
-          if (csvFilepath == null) {
-            //                    isDBFactLoad = true;
-            csvFilepath = meta.getTableName();
-          }
-
-          if (null == measureCol) {
-            measureCol = Arrays.asList(meta.measureColumn);
-          }
-          // Update the Null value comparer and update the String against which we need
-          // to check the values coming from the previous step.
-          logCounter =
-              Integer.parseInt(CarbonCommonConstants.DATA_LOAD_LOG_COUNTER_DEFAULT_COUNTER);
-          if (null != getInputRowMeta()) {
-            meta.updateHierMappings(getInputRowMeta());
-            populateCarbonMeasures(meta.measureColumn);
-            meta.msrMapping = getMeasureOriginalIndexes(meta.measureColumn);
-
-            meta.memberMapping = getMemberMappingOriginalIndexes();
-
-            data.setInputSize(getInputRowMeta().size());
-
-            updatePropMap(meta.actualDimArray);
-            if (meta.isAggregate()) {
-              presentColumnMapIndex = createPresentColumnMapIndexForAggregate();
-            } else {
-              presentColumnMapIndex = createPresentColumnMapIndex();
-
-            }
-            measurePresentMapping = createMeasureMappigs(measureCol);
-            measureSurrogateReqMapping = createMeasureSurrogateReqMapping();
-            createForeignKeyMappingColumns();
-            metaColumnNames = createColumnArrayFromMeta();
-          }
-
-          if (!meta.isAggregate()) {
-            updateHierarchyKeyGenerators(data.getKeyGenerators(), meta.hirches, meta.dimLens,
-                meta.dimColNames);
-          }
-
-          data.setGenerator(
-              KeyGeneratorFactory.getKeyGenerator(getUpdatedLens(meta.dimLens, meta.dimPresent)));
-
-          if (null != getInputRowMeta()) {
-            data.setOutputRowMeta((RowMetaInterface) getInputRowMeta().clone());
-          }
-          this.dimensionColumnIds = meta.getDimensionColumnIds();
-          ColumnsInfo columnsInfo = new ColumnsInfo();
-          columnsInfo.setDims(meta.dims);
-          columnsInfo.setDimColNames(meta.dimColNames);
-          columnsInfo.setKeyGenerators(data.getKeyGenerators());
-          columnsInfo.setDatabaseName(meta.getDatabaseName());
-          columnsInfo.setTableName(meta.getTableName());
-          columnsInfo.setHierTables(meta.hirches.keySet());
-          columnsInfo.setBatchSize(meta.getBatchSize());
-          columnsInfo.setStoreType(meta.getStoreType());
-          columnsInfo.setAggregateLoad(meta.isAggregate());
-          columnsInfo.setMaxKeys(meta.dimLens);
-          columnsInfo.setPropColumns(meta.getPropertiesColumns());
-          columnsInfo.setPropIndx(meta.getPropertiesIndices());
-          columnsInfo.setTimeOrdinalCols(meta.timeOrdinalCols);
-          columnsInfo.setPropTypes(meta.getPropTypes());
-          columnsInfo.setTimDimIndex(meta.timeDimeIndex);
-          columnsInfo.setDimHierRel(meta.getDimTableArray());
-          columnsInfo.setBaseStoreLocation(getCarbonLocalBaseStoreLocation());
-          columnsInfo.setTableName(meta.getTableName());
-          columnsInfo.setPrimaryKeyMap(meta.getPrimaryKeyMap());
-          columnsInfo.setMeasureColumns(meta.measureColumn);
-          columnsInfo.setComplexTypesMap(meta.getComplexTypes());
-          columnsInfo.setDimensionColumnIds(this.dimensionColumnIds);
-          columnsInfo.setColumnSchemaDetailsWrapper(meta.getColumnSchemaDetailsWrapper());
-          columnsInfo.setColumnProperties(meta.getColumnPropertiesMap());
-          updateBagLogFileName();
-          columnsInfo.setTimeOrdinalIndices(meta.timeOrdinalIndices);
-          surrogateKeyGen = new FileStoreSurrogateKeyGenForCSV(columnsInfo, meta.getPartitionID(),
-              meta.getSegmentId(), meta.getTaskNo());
-          data.setSurrogateKeyGen(surrogateKeyGen);
-          updateStoreLocation();
-
-          // Check the insert hierarchies required or not based on that
-          // Create the list which will hold the hierarchies required to be created
-          // i.e. denormalized hierarchies.
-          if (null != getInputRowMeta()) {
-            denormHierarchies = getDenormalizedHierarchies();
-          }
-
-          if (null != getInputRowMeta()) {
-            // We consider that there is no time dimension,in these case
-            // the
-            // timeIndex = -1
-
-            ValueMetaInterface[] out = null;
-            out = new ValueMetaInterface[meta.normLength + meta.msrMapping.length];
-            int outCounter = 0;
-            for (int i = 0; i < meta.actualDimArray.length; i++) {
-              if (meta.dimPresent[i]) {
-                ValueMetaInterface x =
-                    new ValueMeta(meta.actualDimArray[i], ValueMetaInterface.TYPE_STRING,
-                        ValueMetaInterface.STORAGE_TYPE_BINARY_STRING);
-                x.setStorageMetadata(
-                    (new ValueMeta(meta.actualDimArray[i], ValueMetaInterface.TYPE_STRING,
-                        ValueMetaInterface.STORAGE_TYPE_NORMAL)));
-                x.setStringEncoding(BYTE_ENCODING);
-                x.setStringEncoding(BYTE_ENCODING);
-                x.getStorageMetadata().setStringEncoding(BYTE_ENCODING);
-
-                out[outCounter] = x;
-                outCounter++;
-              }
-            }
-
-            for (int j = 0; j < meta.measureColumn.length; j++) {
-              for (int k = 0; k < data.getOutputRowMeta().size(); k++) {
-                if (meta.measureColumn[j]
-                    .equalsIgnoreCase(data.getOutputRowMeta().getValueMeta(k).getName())) {
-                  out[outCounter] =
-                      new ValueMeta(meta.measureColumn[j], ValueMetaInterface.TYPE_NUMBER,
-                          ValueMetaInterface.STORAGE_TYPE_NORMAL);
-                  out[outCounter].setStorageMetadata(
-                      new ValueMeta(meta.measureColumn[j], ValueMetaInterface.TYPE_NUMBER,
-                          ValueMetaInterface.STORAGE_TYPE_NORMAL));
-                  outCounter++;
-                  break;
-                }
-              }
-            }
-            data.getOutputRowMeta().setValueMetaList(Arrays.asList(out));
-          }
-        }
-        columnSchemaDetailsWrapper = meta.getColumnSchemaDetailsWrapper();
-        if (null != getInputRowMeta()) {
-          generateNoDictionaryAndComplexIndexMapping();
-          data.getSurrogateKeyGen()
-              .setDimensionOrdinalToDimensionMapping(populateNameToCarbonDimensionMap());
-        }
-        serializationNullFormat =
-            meta.getTableOptionWrapper().get(SERIALIZATION_NULL_FORMAT.getName());
-        boolean badRecordsLoggerEnable;
-        boolean badRecordsLogRedirect = false;
-        boolean badRecordConvertNullDisable = false;
-        boolean isDataLoadFail = false;
-        badRecordsLoggerEnable = Boolean
-            .parseBoolean(meta.getTableOptionWrapper().get(BAD_RECORDS_LOGGER_ENABLE.getName()));
-        String bad_records_action =
-            meta.getTableOptionWrapper().get(BAD_RECORDS_ACTION.getName());
-        if (null != bad_records_action) {
-          LoggerAction loggerAction = null;
-          try {
-            loggerAction = LoggerAction.valueOf(bad_records_action.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;
-          }
-        }
-        String key = meta.getDatabaseName() + '/' + meta.getTableName() +
-            '_' + meta.getTableName();
-        badRecordsLogger = new BadRecordsLogger(key, csvFilepath, getBadLogStoreLocation(
-            meta.getDatabaseName() + '/' + meta.getTableName() + "/" + meta.getTaskNo()),
-            badRecordsLogRedirect, badRecordsLoggerEnable, badRecordConvertNullDisable,
-            isDataLoadFail);
-        HashMap<String, String> dateformatsHashMap = new HashMap<String, String>();
-        if (meta.dateFormat != null) {
-          String[] dateformats = meta.dateFormat.split(CarbonCommonConstants.COMMA);
-          for (String dateFormat:dateformats) {
-            String[] dateFormatSplits = dateFormat.split(":", 2);
-            dateformatsHashMap.put(dateFormatSplits[0].toLowerCase().trim(),
-                dateFormatSplits[1].trim());
-          }
-        }
-        String[] DimensionColumnIds = meta.getDimensionColumnIds();
-        directDictionaryGenerators =
-            new DirectDictionaryGenerator[DimensionColumnIds.length];
-        for (int i = 0; i < DimensionColumnIds.length; i++) {
-          ColumnSchemaDetails columnSchemaDetails = columnSchemaDetailsWrapper.get(
-              DimensionColumnIds[i]);
-          if (columnSchemaDetails.isDirectDictionary()) {
-            String columnName = columnSchemaDetails.getColumnName();
-            DataType columnType = columnSchemaDetails.getColumnType();
-            if (dateformatsHashMap.containsKey(columnName)) {
-              directDictionaryGenerators[i] =
-                  DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
-                      columnType, dateformatsHashMap.get(columnName));
-            } else {
-              directDictionaryGenerators[i] =
-                  DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(columnType);
-            }
-          }
-        }
-      }
-      // no more input to be expected...
-      if (r == null) {
-        return processWhenRowIsNull();
-      }
-      // proecess the first
-      Object[] out = process(r);
-      readCounter++;
-      if (null != out) {
-        writeCounter++;
-        putRow(data.getOutputRowMeta(), out);
-      }
-      // start multi-thread to process
-      int numberOfNodes;
-      try {
-        numberOfNodes = Integer.parseInt(CarbonProperties.getInstance()
-            .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-                CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-      } catch (NumberFormatException exc) {
-        numberOfNodes = NUM_CORES_DEFAULT_VAL;
-      }
-
-      startReadingProcess(numberOfNodes);
-      badRecordsLogger.closeStreams();
-      if (!meta.isAggregate()) {
-        closeNormalizedHierFiles();
-      }
-      if (writeCounter == 0) {
-        return processWhenRowIsNull();
-      }
-      CarbonUtil.writeLevelCardinalityFile(loadFolderLoc, meta.getTableName(),
-          getUpdatedCardinality());
-      LOGGER.info("Record Procerssed For table: " + meta.getTableName());
-      String logMessage =
-          "Summary: Carbon CSV Based Seq Gen Step : " + readCounter + ": Write: " + writeCounter;
-      LOGGER.info(logMessage);
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordGeneratingDictionaryValuesTime(
-          meta.getPartitionID(), System.currentTimeMillis());
-      setOutputDone();
-
-    } catch (RuntimeException ex) {
-      LOGGER.error(ex);
-      throw ex;
-    } catch (Exception ex) {
-      LOGGER.error(ex);
-      throw new RuntimeException(ex);
-    }
-    return false;
-  }
-
-  private void generateNoDictionaryAndComplexIndexMapping() {
-    isNoDictionaryColumn = new boolean[metaColumnNames.length];
-    isComplexTypeColumn = new boolean[metaColumnNames.length];
-    noDictionaryAndComplexIndexMapping = new int[metaColumnNames.length];
-    isStringDataType = new boolean[metaColumnNames.length];
-    dataTypes = new String[metaColumnNames.length];
-    complexTypes = new GenericDataType[meta.getComplexTypeColumns().length];
-    for (int i = 0; i < meta.noDictionaryCols.length; i++) {
-      for (int j = 0; j < metaColumnNames.length; j++) {
-        if (CarbonCommonConstants.STRING
-            .equalsIgnoreCase(meta.dimColDataTypes.get(metaColumnNames[j]))) {
-          isStringDataType[j] = true;
-        }
-        dataTypes[j] = meta.dimColDataTypes.get(metaColumnNames[j].toLowerCase());
-        if (meta.noDictionaryCols[i].equalsIgnoreCase(
-            meta.getTableName() + CarbonCommonConstants.UNDERSCORE + metaColumnNames[j])) {
-          isNoDictionaryColumn[j] = true;
-          noDictionaryAndComplexIndexMapping[j] = i;
-          break;
-        }
-      }
-    }
-    for (int i = 0; i < meta.getComplexTypeColumns().length; i++) {
-      for (int j = 0; j < metaColumnNames.length; j++) {
-        if (meta.getComplexTypeColumns()[i].equalsIgnoreCase(metaColumnNames[j])) {
-          isComplexTypeColumn[j] = true;
-          complexTypes[i] = meta.complexTypes.get(meta.getComplexTypeColumns()[i]);
-          noDictionaryAndComplexIndexMapping[j] = i + meta.noDictionaryCols.length;
-          break;
-        }
-      }
-    }
-  }
-
-  private void startReadingProcess(int numberOfNodes) throws KettleException, InterruptedException {
-    startProcess(numberOfNodes);
-  }
-
-  private boolean processWhenRowIsNull() throws KettleException {
-    // If first request itself is null then It will not enter the first block and
-    // in data surrogatekeygen will not be initialized so it can throw NPE.
-    if (data.getSurrogateKeyGen() == null) {
-      setOutputDone();
-      LOGGER.info("Record Procerssed For table: " + meta.getTableName());
-      String logMessage =
-          "Summary: Carbon CSV Based Seq Gen Step:  Read: " + readCounter + ": Write: "
-              + writeCounter;
-      LOGGER.info(logMessage);
-      return false;
-    }
-
-    setOutputDone();
-    LOGGER.info("Record Processed For table: " + meta.getTableName());
-    String logMessage =
-        "Summary: Carbon CSV Based Seq Gen Step:  Read: " + readCounter + ": Write: "
-            + writeCounter;
-    LOGGER.info(logMessage);
-    return false;
-  }
-
-  /**
-   * holds the value to be considered as null while dataload
-   */
-  private String serializationNullFormat;
-
-  private List<String> getDenormalizedHierarchies() {
-    List<String> hierList = Arrays.asList(meta.hierNames);
-    List<String> denormHiers = new ArrayList<String>(10);
-    for (Iterator<Entry<String, int[]>> iterator = meta.hirches.entrySet().iterator(); iterator
-        .hasNext(); ) {
-      Entry<String, int[]> entry = iterator.next();
-      String name = entry.getKey();
-
-      if (hierList.contains(name)) {
-        continue;
-      } else if (entry.getValue().length > 1) {
-        denormHiers.add(name);
-      }
-    }
-
-    return denormHiers;
-  }
-
-  private void updatePropMap(String[] actualDimArray) {
-    if (null == propMap) {
-      propMap = new HashMap<String, int[]>(actualDimArray.length);
-    }
-    List<String> currentColNames = new ArrayList<String>(10);
-    for (int i = 0; i < getInputRowMeta().size(); i++) {
-      currentColNames.add(getInputRowMeta().getValueMeta(i).getName());
-    }
-
-    List<String> currentColName = new ArrayList<String>(actualDimArray.length);
-
-    for (int i = 0; i < getInputRowMeta().size(); i++) {
-      String columnName = getInputRowMeta().getValueMeta(i).getName();
-      String hier = meta.foreignKeyHierarchyMap.get(columnName);
-      if (null != hier) {
-        if (hier.indexOf(CarbonCommonConstants.COMA_SPC_CHARACTER) > -1) {
-          String[] splittedHiers = hier.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-          for (String hierName : splittedHiers) {
-            String tableName = meta.getHierDimTableMap().get(hier);
-            String[] cols = meta.hierColumnMap.get(hierName);
-            if (null != cols) {
-              for (String column : cols) {
-                currentColName.add(tableName + '_' + column);
-              }
-            }
-          }
-        } else {
-          String tableName = meta.getHierDimTableMap().get(hier);
-
-          String[] columns = meta.hierColumnMap.get(hier);
-
-          if (null != columns) {
-            for (String column : columns) {
-              currentColName.add(tableName + '_' + column);
-            }
-          }
-        }
-      } else
-      // then it can be direct column name if not foreign key.
-      {
-        currentColName.add(meta.getTableName() + '_' + columnName);
-      }
-    }
-
-    String[] currentColNamesArray = currentColName.toArray(new String[currentColName.size()]);
-
-    List<HierarchiesInfo> metahierVoList = meta.getMetahierVoList();
-
-    if (null == metahierVoList) {
-      return;
-    }
-    for (HierarchiesInfo hierInfo : metahierVoList) {
-
-      Map<String, String[]> columnPropMap = hierInfo.getColumnPropMap();
-
-      Set<Entry<String, String[]>> entrySet = columnPropMap.entrySet();
-
-      for (Entry<String, String[]> entry : entrySet) {
-        String[] propColmns = entry.getValue();
-        int[] index = getIndex(currentColNamesArray, propColmns);
-        propMap.put(entry.getKey(), index);
-      }
-    }
-
-  }
-
-  private int[] getIndex(String[] currentColNamesArray, String[] propColmns) {
-    int[] resultIndex = new int[propColmns.length];
-
-    for (int i = 0; i < propColmns.length; i++) {
-      for (int j = 0; j < currentColNamesArray.length; j++) {
-        if (propColmns[i].equalsIgnoreCase(currentColNamesArray[j])) {
-          resultIndex[i] = j;
-          break;
-        }
-      }
-    }
-
-    return resultIndex;
-  }
-
-  private void closeNormalizedHierFiles() throws KettleException {
-    if (null == filemanager) {
-      return;
-    }
-    int hierLen = filemanager.size();
-
-    for (int i = 0; i < hierLen; i++) {
-      FileData hierFileData = (FileData) filemanager.get(i);
-      String hierInProgressFileName = hierFileData.getFileName();
-      HierarchyValueWriterForCSV hierarchyValueWriter =
-          nrmlizedHierWriterMap.get(hierInProgressFileName);
-      if (null == hierarchyValueWriter) {
-        continue;
-      }
-
-      List<ByteArrayHolder> holders = hierarchyValueWriter.getByteArrayList();
-      Collections.sort(holders);
-
-      for (ByteArrayHolder holder : holders) {
-        hierarchyValueWriter.writeIntoHierarchyFile(holder.getMdKey(), holder.getPrimaryKey());
-      }
-
-      // now write the byte array in the file.
-      FileChannel bufferedOutStream = hierarchyValueWriter.getBufferedOutStream();
-      if (null == bufferedOutStream) {
-        continue;
-      }
-      CarbonUtil.closeStreams(bufferedOutStream);
-
-      hierInProgressFileName = hierFileData.getFileName();
-      int counter = hierarchyValueWriter.getCounter();
-      String storePath = hierFileData.getStorePath();
-      String changedFileName = hierInProgressFileName + (counter - 1);
-      hierInProgressFileName = changedFileName + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-
-      File currentFile = new File(storePath + File.separator + hierInProgressFileName);
-      File destFile = new File(storePath + File.separator + changedFileName);
-      if (currentFile.exists()) {
-        boolean renameTo = currentFile.renameTo(destFile);
-
-        if (!renameTo) {
-          LOGGER.info("Not Able to Rename File : " + currentFile.getName());
-        }
-      }
-
-    }
-
-  }
-
-  /**
-   * Load Store location
-   */
-  private void updateStoreLocation() {
-    loadFolderLoc = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(meta.getDatabaseName(), meta.getTableName(), meta.getTaskNo(),
-            meta.getPartitionID(), meta.getSegmentId() + "", false);
-  }
-
-  private String getBadLogStoreLocation(String storeLocation) {
-    String badLogStoreLocation =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC);
-    badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation;
-
-    return badLogStoreLocation;
-  }
-
-  private void updateBagLogFileName() {
-    csvFilepath = new File(csvFilepath).getName();
-    if (csvFilepath.indexOf(".") > -1) {
-      csvFilepath = csvFilepath.substring(0, csvFilepath.indexOf("."));
-    }
-
-    csvFilepath = csvFilepath + '_' + System.currentTimeMillis();
-
-  }
-
-  private void startProcess(final int numberOfNodes) throws RuntimeException {
-    exec = Executors.newFixedThreadPool(numberOfNodes);
-
-    Callable<Void> callable = new Callable<Void>() {
-      @Override public Void call() throws RuntimeException {
-        StandardLogService
-            .setThreadName(StandardLogService.getPartitionID(meta.getTableName()), null);
-        try {
-            doProcess();
-        } catch (Throwable e) {
-          LOGGER.error(e, "Thread is terminated due to error");
-          threadStatusObserver.notifyFailed(e);
-        }
-        return null;
-      }
-    };
-    List<Future<Void>> results = new ArrayList<Future<Void>>(10);
-    for (int i = 0; i < numberOfNodes; i++) {
-      results.add(exec.submit(callable));
-    }
-
-    this.resultArray = results.toArray(new Future[results.size()]);
-    try {
-      for (int j = 0; j < this.resultArray.length; j++) {
-        this.resultArray[j].get();
-      }
-    } catch (InterruptedException | ExecutionException e) {
-      throw new RuntimeException("Thread InterruptedException", e);
-    } finally {
-      exec.shutdownNow();
-    }
-  }
-
-  private int[] getUpdatedLens(int[] lens, boolean[] presentDims) {
-    int k = 0;
-    int[] integers = new int[meta.normLength];
-    for (int i = 0; i < lens.length; i++) {
-      if (presentDims[i]) {
-        integers[k] = lens[i];
-        k++;
-      }
-    }
-    return integers;
-  }
-
-  /**
-   * @return
-   */
-  private int[] getUpdatedCardinality() {
-    int[] maxSurrogateKeyArray = data.getSurrogateKeyGen().max;
-
-    List<Integer> dimCardWithComplex = new ArrayList<Integer>();
-
-    for (int i = 0; i < meta.dimColNames.length; i++) {
-      GenericDataType complexDataType =
-          meta.complexTypes.get(meta.dimColNames[i].substring(meta.getTableName().length() + 1));
-      if (complexDataType != null) {
-        complexDataType.fillCardinalityAfterDataLoad(dimCardWithComplex, maxSurrogateKeyArray);
-      } else {
-        dimCardWithComplex.add(maxSurrogateKeyArray[i]);
-      }
-    }
-
-    int[] complexDimCardinality = new int[dimCardWithComplex.size()];
-    for (int i = 0; i < dimCardWithComplex.size(); i++) {
-      complexDimCardinality[i] = dimCardWithComplex.get(i);
-    }
-    return complexDimCardinality;
-  }
-
-  private void doProcess() throws RuntimeException {
-    try {
-      for (DirectDictionaryGenerator directDictionaryGenerator: directDictionaryGenerators) {
-        if (directDictionaryGenerator != null) {
-          directDictionaryGenerator.initialize();
-        }
-      }
-
-      while (true) {
-        Object[] r = null;
-        synchronized (getRowLock) {
-
-          r = getRow();
-          readCounter++;
-        }
-
-        // no more input to be expected...
-        if (r == null) {
-          readCounter--;
-          break;
-        }
-        Object[] out = process(r);
-        if (null == out) {
-          continue;
-        }
-
-        synchronized (putRowLock) {
-          putRow(data.getOutputRowMeta(), out);
-          processRecord();
-          writeCounter++;
-        }
-      }
-    } catch (RuntimeException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private void processRecord() {
-    if (readCounter % logCounter == 0) {
-      LOGGER.info("Record Procerssed For table: " + meta.getTableName());
-      String logMessage = "Carbon Csv Based Seq Gen Step: Record Read : " + readCounter;
-      LOGGER.info(logMessage);
-    }
-  }
-
-  private String getCarbonLocalBaseStoreLocation() {
-    String tempLocationKey =
-        meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName()
-            + CarbonCommonConstants.UNDERSCORE + meta.getTaskNo();
-    String strLoc = CarbonProperties.getInstance()
-        .getProperty(tempLocationKey, CarbonCommonConstants.STORE_LOCATION_DEFAULT_VAL);
-    File f = new File(strLoc);
-    String absoluteStorePath = f.getAbsolutePath();
-    return absoluteStorePath;
-  }
-
-  private Object[] process(Object[] r) throws RuntimeException {
-    try {
-      Object[] out = populateOutputRow(r);
-      if (out != null) {
-        for (int i = 0; i < meta.normLength - meta.complexTypes.size(); i++) {
-          if (null == NonDictionaryUtil.getDimension(i, out)) {
-            NonDictionaryUtil.setDimension(i, 1, out);
-          }
-        }
-      }
-      return out;
-
-    } catch (KettleException e) {
-      throw new RuntimeException(e);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  private Object[] populateOutputRow(Object[] r) throws KettleException {
-
-    // Copy the dimension String values to output
-    int[] memberMapping = meta.memberMapping;
-    int inputColumnsSize = metaColumnNames.length;
-    boolean isGenerated = false;
-    int generatedSurrogate = -1;
-
-    //If CSV Exported from DB and we enter one row down then that row become empty.
-    // In that case it will have first value empty and other values will be null
-    // So If records is coming like this then we need to write this records as a bad Record.
-
-    if (null == r[0] && badRecordsLogger.isBadRecordConvertNullDisable()) {
-      badRecordsLogger
-          .addBadRecordsToBuilder(r, "Column Names are coming NULL");
-      return null;
-    }
-
-    Map<String, Dictionary> dictionaryCaches = surrogateKeyGen.getDictionaryCaches();
-    Object[] out =
-        new Object[meta.normLength + meta.msrs.length - meta.complexTypes.size()];
-    int dimLen = meta.dims.length;
-
-    Object[] newArray = new Object[CarbonCommonConstants.ARRAYSIZE];
-
-    ByteBuffer[] byteBufferArr = null;
-    if (null != meta.noDictionaryCols) {
-      byteBufferArr = new ByteBuffer[meta.noDictionaryCols.length + meta.complexTypes.size()];
-    }
-    int i = 0;
-    int index = 0;
-    int l = 0;
-    int msrCount = 0;
-    boolean isNull = false;
-    for (int j = 0; j < inputColumnsSize; j++) {
-      String columnName = metaColumnNames[j];
-      String foreignKeyColumnName = foreignKeyMappingColumns[j];
-      // check if it is ignore dictionary dimension or not . if yes directly write byte buffer
-      String tuple = null == r[j] ?
-          CarbonCommonConstants.MEMBER_DEFAULT_VAL :
-          (String) r[j];
-      // check whether the column value is the value to be  serialized as null.
-      boolean isSerialized = false;
-      if (tuple.equalsIgnoreCase(serializationNullFormat)) {
-        tuple = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
-        isSerialized = true;
-      }
-      if (isNoDictionaryColumn[j]) {
-        String dimensionValue =
-            processnoDictionaryDim(noDictionaryAndComplexIndexMapping[j], tuple, dataTypes[j],
-                isStringDataType[j], byteBufferArr);
-        if (!isSerialized && !isStringDataType[j] && CarbonCommonConstants.MEMBER_DEFAULT_VAL
-            .equals(dimensionValue)) {
-          failDataLoad(r, index, columnName, msrDataType[meta.msrMapping[msrCount]].name());
-          addEntryToBadRecords(r, j, columnName, dataTypes[j]);
-          if (badRecordsLogger.isBadRecordConvertNullDisable()) {
-            return null;
-          }
-        }
-        continue;
-      }
-      // There is a possibility that measure can be referred as dimensions also
-      // so in that case we need to just copy the value into the measure column index.
-      //if it enters here means 3 possibility
-      //1) this is not foreign key it can be direct columns
-      //2) This column present in the csv file but in the schema it is not present.
-      //3) This column can be measure column
-
-      if (measurePresentMapping[j]) {
-        String msr = tuple == null ? null : tuple.toString();
-        isNull = CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(msr);
-        if (measureSurrogateReqMapping[j] && !isNull) {
-          Integer surrogate = 0;
-          if (null == foreignKeyColumnName) {
-            // If foreignKeyColumnName is null till here that means this
-            // measure column is of type count and data type may be string
-            // so we have to create the surrogate key for the values.
-            surrogate = createSurrogateForMeasure(msr, columnName);
-            if (presentColumnMapIndex[j] > -1) {
-              isGenerated = true;
-              generatedSurrogate = surrogate;
-            }
-          } else {
-            surrogate = surrogateKeyGen.generateSurrogateKeys(msr, foreignKeyColumnName);
-          }
-
-          out[memberMapping[dimLen + index]] = surrogate.doubleValue();
-        } else if (!isSerialized &&  (isNull || msr == null
-            || msr.length() == 0)) {
-          failDataLoad(r, index, columnName,
-              msrDataType[meta.msrMapping[msrCount]].name());
-          addEntryToBadRecords(r, j, columnName,
-              msrDataType[meta.msrMapping[msrCount]].name());
-          if (badRecordsLogger.isBadRecordConvertNullDisable()) {
-            return null;
-          }
-        } else {
-          try {
-            if (!isNull && null != msr && msr.length() > 0) {
-              Object measureValueBasedOnDataType = DataTypeUtil
-                  .getMeasureValueBasedOnDataType(msr, msrDataType[meta.msrMapping[msrCount]],
-                      meta.carbonMeasures[meta.msrMapping[msrCount]]);
-              if (null == measureValueBasedOnDataType) {
-                addEntryToBadRecords(r, j, columnName,
-                    msrDataType[meta.msrMapping[msrCount]].name());
-                if (badRecordsLogger.isBadRecordConvertNullDisable()) {
-                  return null;
-                }
-                LOGGER.warn("Cannot convert : " + msr
-                    + " to Numeric type value. Value considered as null.");
-              }
-              out[memberMapping[dimLen + index] - meta.complexTypes.size()] =
-                  measureValueBasedOnDataType;
-            }
-          } catch (NumberFormatException e) {
-            failDataLoad(r, index, columnName,
-                msrDataType[meta.msrMapping[msrCount]].name());
-            addEntryToBadRecords(r, j, columnName, msrDataType[meta.msrMapping[msrCount]].name());
-            if (badRecordsLogger.isBadRecordConvertNullDisable()) {
-              return null;
-            }
-            LOGGER.warn(
-                "Cannot convert : " + msr + " to Numeric type value. Value considered as null.");
-            out[memberMapping[dimLen + index] - meta.complexTypes.size()] = null;
-          }
-        }
-
-        index++;
-        msrCount++;
-        if (presentColumnMapIndex[j] < 0 && null == foreignKeyColumnName) {
-          continue;
-        }
-      }
-
-      boolean isPresentInSchema = false;
-      if (null == foreignKeyColumnName) {
-        //if it enters here means 3 possibility
-        //1) this is not foreign key it can be direct columns
-        //2) This column present in the csv file but in the schema it is not present.
-        //3) This column can be measure column
-        int m = presentColumnMapIndex[j];
-        if (m >= 0) {
-          isPresentInSchema = true;
-        }
-
-        if (isPresentInSchema) {
-          foreignKeyColumnName = meta.dimColNames[m];
-        } else {
-          continue;
-        }
-      }
-
-      //If it refers to multiple hierarchy by same foreign key
-      if (foreignKeyMappingColumnsForMultiple[j] != null) {
-        String[] splittedHiers = foreignKeyMappingColumnsForMultiple[j];
-
-        for (String hierForignKey : splittedHiers) {
-          Dictionary dicCache = dictionaryCaches.get(hierForignKey);
-
-          String actualHierName = null;
-          if (!isPresentInSchema) {
-            actualHierName = meta.hierNames[l++];
-
-          }
-
-          Map<Integer, int[]> cache = surrogateKeyGen.getHierCache().get(actualHierName);
-          int[] surrogateKeyForHierarchy = null;
-          if (null != cache) {
-
-            Integer keyFromCsv = dicCache.getSurrogateKey(tuple);
-
-            if (null != keyFromCsv) {
-              surrogateKeyForHierarchy = cache.get(keyFromCsv);
-            } else {
-              addMemberNotExistEntry(r, j, columnName);
-              return null;
-            }
-            // If cardinality exceeded for some levels then
-            // for that hierarchy will not be their
-            // so while joining with fact table if we are
-            // getting this scenerio we will log it
-            // in bad records
-            if (null == surrogateKeyForHierarchy) {
-              addEntryToBadRecords(r, j, columnName);
-              return null;
-
-            }
-          } else {
-            surrogateKeyForHierarchy = new int[1];
-            surrogateKeyForHierarchy[0] =
-                surrogateKeyGen.generateSurrogateKeys(tuple, foreignKeyColumnName);
-          }
-          for (int k = 0; k < surrogateKeyForHierarchy.length; k++) {
-            if (dimPresentCsvOrder[i]) {
-              out[memberMapping[i]] = surrogateKeyForHierarchy[k];
-            }
-
-            i++;
-          }
-
-        }
-
-      } else if (isComplexTypeColumn[j]) {
-        //If it refers to single hierarchy
-        try {
-          GenericDataType complexType =
-              complexTypes[noDictionaryAndComplexIndexMapping[j] - meta.noDictionaryCols.length];
-          ByteArrayOutputStream byteArray = new ByteArrayOutputStream();
-          DataOutputStream dataOutputStream = new DataOutputStream(byteArray);
-          complexType.parseStringAndWriteByteArray(meta.getTableName(), tuple,
-              new String[] { meta.getComplexDelimiterLevel1(), meta.getComplexDelimiterLevel2() },
-              0, dataOutputStream, surrogateKeyGen);
-          byteBufferArr[noDictionaryAndComplexIndexMapping[j]] =
-              ByteBuffer.wrap(byteArray.toByteArray());
-          if (null != byteArray) {
-            byteArray.close();
-          }
-        } catch (IOException e1) {
-          throw new KettleException(
-              "Parsing complex string and generating surrogates/ByteArray failed. ", e1);
-        }
-        i++;
-      } else {
-        Dictionary dicCache = dictionaryCaches.get(foreignKeyColumnName);
-
-        String actualHierName = null;
-        if (!isPresentInSchema) {
-          actualHierName = meta.hierNames[l++];
-
-        }
-
-        Map<Integer, int[]> cache = surrogateKeyGen.getHierCache().get(actualHierName);
-        int[] surrogateKeyForHrrchy = null;
-        if (null != cache) {
-          Integer keyFromCsv = dicCache.getSurrogateKey(tuple);
-
-          if (null != keyFromCsv) {
-            surrogateKeyForHrrchy = cache.get(keyFromCsv);
-          } else {
-            addMemberNotExistEntry(r, j, columnName);
-            return null;
-          }
-          // If cardinality exceeded for some levels then for that hierarchy will not be their
-          // so while joining with fact table if we are getting this scenerio we will log it
-          // in bad records
-          if (null == surrogateKeyForHrrchy) {
-            addEntryToBadRecords(r, j, columnName);
-            return null;
-
-          }
-        } else {
-          int[] propIndex = propMap.get(foreignKeyColumnName);
-          Object[] properties;
-          if (null == propIndex) {
-            properties = new Object[0];
-          } else {
-            properties = new Object[propIndex.length];
-            for (int ind = 0; ind < propIndex.length; ind++) {
-              Object objectValue = r[propIndex[ind]];
-              properties[ind] = null == objectValue ?
-                  CarbonCommonConstants.MEMBER_DEFAULT_VAL : (String)objectValue;
-            }
-          }
-          surrogateKeyForHrrchy = new int[1];
-          if (isGenerated && !isNull) {
-            surrogateKeyForHrrchy[0] = generatedSurrogate;
-            isGenerated = false;
-            generatedSurrogate = -1;
-          } else {
-            int m = j;
-            if (isPresentInSchema) {
-              m = presentColumnMapIndex[j];
-            }
-            ColumnSchemaDetails details = columnSchemaDetailsWrapper.get(dimensionColumnIds[m]);
-            if (details.isDirectDictionary()) {
-              surrogateKeyForHrrchy[0] =
-                  directDictionaryGenerators[m].generateDirectSurrogateKey(tuple);
-              if (!isSerialized && surrogateKeyForHrrchy[0] == 1) {
-                failDataLoad(r, index, columnName, details.getColumnType().getName());
-                addEntryToBadRecords(r, j, columnName, details.getColumnType().name());
-                if (badRecordsLogger.isBadRecordConvertNullDisable()) {
-                  return null;
-                }
-              }
-              surrogateKeyGen.max[m] = Integer.MAX_VALUE;
-
-            } else {
-              String parsedValue = DataTypeUtil.parseValue(tuple, data.getSurrogateKeyGen()
-                  .getDimensionOrdinalToDimensionMapping()[memberMapping[i]]);
-              if (null == parsedValue) {
-                surrogateKeyForHrrchy[0] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY;
-              } else {
-                surrogateKeyForHrrchy[0] =
-                    surrogateKeyGen.generateSurrogateKeys(parsedValue, foreignKeyColumnName);
-              }
-            }
-          }
-          if (surrogateKeyForHrrchy[0] == CarbonCommonConstants.INVALID_SURROGATE_KEY) {
-
-            if (!isSerialized) {
-              int m = j;
-              if (isPresentInSchema) {
-                m = presentColumnMapIndex[j];
-              }
-              ColumnSchemaDetails details = columnSchemaDetailsWrapper.get(dimensionColumnIds[m]);
-              failDataLoad(r, index, columnName, details.getColumnType().getName());
-              addEntryToBadRecords(r, j, columnName);
-              if (badRecordsLogger.isBadRecordConvertNullDisable()) {
-                return null;
-              }
-            }
-            surrogateKeyForHrrchy[0] = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY;
-          }
-        }
-        for (int k = 0; k < surrogateKeyForHrrchy.length; k++) {
-          if (dimPresentCsvOrder[i]) {
-            if (duplicateColMapping[j] != null) {
-              for (int m = 0; m < duplicateColMapping[j].length; m++) {
-                out[duplicateColMapping[j][m]] = Integer.valueOf(surrogateKeyForHrrchy[k]);
-              }
-            } else {
-              out[memberMapping[i]] = Integer.valueOf(surrogateKeyForHrrchy[k]);
-            }
-          }
-
-          i++;
-        }
-      }
-    }
-
-    insertHierIfRequired(out);
-    NonDictionaryUtil
-        .prepareOut(newArray, byteBufferArr, out, dimLen - meta.complexTypes.size());
-
-    return newArray;
-  }
-
-  private void failDataLoad(Object[] row, int index, String columnName, String dataType)
-      throws KettleException {
-    if (badRecordsLogger.isDataLoadFail()) {
-      String errorMessage = getBadRecordEntry(row, index, columnName, dataType);
-      dis.setVariable(CarbonCommonConstants.BAD_RECORD_KEY, errorMessage);
-      LOGGER.error("Data load failed due to bad record. " + errorMessage);
-      throw new KettleException("Data load failed due to bad record");
-    }
-  }
-
-  private void addEntryToBadRecords(Object[] r, int j, String columnName, String dataType) {
-    dataType = DataTypeUtil.getColumnDataTypeDisplayName(dataType);
-    badRecordsLogger.addBadRecordsToBuilder(r,
-        "The value " + " \"" + r[j] + "\"" + " with column name " + columnName
-            + " and column data type " + dataType + " is not a valid " + dataType + " type.");
-  }
-
-  private String getBadRecordEntry(Object[] r, int j, String columnName, String dataType) {
-    dataType = DataTypeUtil.getColumnDataTypeDisplayName(dataType);
-    String badRecord = "The value " + " \"" + r[j] + "\"" + " with column name " + columnName
-        + " and column data type " + dataType + " is not a valid Record";
-    return badRecord;
-  }
-
-  private void addEntryToBadRecords(Object[] r, int j, String columnName) {
-    badRecordsLogger.addBadRecordsToBuilder(r,
-        "Surrogate key for value " + " \"" + r[j] + "\"" + " with column name " + columnName
-            + " not found in dictionary cache");
-  }
-
-  private void addMemberNotExistEntry(Object[] r, int j, String columnName) {
-    badRecordsLogger.addBadRecordsToBuilder(r,
-        "For Coulmn " + columnName + " \"" + r[j] + "\""
-            + " member not exist in the dimension table ");
-  }
-
-  private void insertHierIfRequired(Object[] out) throws KettleException {
-    if (denormHierarchies.size() > 0) {
-      insertHierarichies(out);
-    }
-  }
-
-  private int[] createPresentColumnMapIndex() {
-    int[] presentColumnMapIndex = new int[getInputRowMeta().size()];
-    duplicateColMapping = new int[getInputRowMeta().size()][];
-    Arrays.fill(presentColumnMapIndex, -1);
-    for (int j = 0; j < getInputRowMeta().size(); j++) {
-      String columnName = getInputRowMeta().getValueMeta(j).getName();
-
-      int m = 0;
-
-      String foreignKey = meta.foreignKeyHierarchyMap.get(columnName);
-      if (foreignKey == null) {
-        List<Integer> repeats = new ArrayList<Integer>(10);
-        for (String col : meta.dimColNames) {
-          if (col.equalsIgnoreCase(meta.getTableName() + '_' + columnName)) {
-            presentColumnMapIndex[j] = m;
-            repeats.add(m);
-          }
-          m++;
-        }
-        if (repeats.size() > 1) {
-          int[] dims = new int[repeats.size()];
-          for (int i = 0; i < dims.length; i++) {
-            dims[i] = repeats.get(i);
-          }
-          duplicateColMapping[j] = dims;
-        }
-
-      } else {
-        for (String col : meta.actualDimArray) {
-          if (col.equalsIgnoreCase(columnName)) {
-            presentColumnMapIndex[j] = m;
-            break;
-          }
-          m++;
-        }
-
-      }
-    }
-    return presentColumnMapIndex;
-  }
-
-  private int[] createPresentColumnMapIndexForAggregate() {
-    int[] presentColumnMapIndex = new int[getInputRowMeta().size()];
-    duplicateColMapping = new int[getInputRowMeta().size()][];
-    Arrays.fill(presentColumnMapIndex, -1);
-    for (int j = 0; j < getInputRowMeta().size(); j++) {
-      String columnName = getInputRowMeta().getValueMeta(j).getName();
-
-      int m = 0;
-
-      String foreignKey = meta.foreignKeyHierarchyMap.get(columnName);
-      if (foreignKey == null) {
-        for (String col : meta.actualDimArray) {
-          if (col.equalsIgnoreCase(columnName)) {
-            presentColumnMapIndex[j] = m;
-            break;
-          }
-          m++;
-        }
-      }
-    }
-    return presentColumnMapIndex;
-  }
-
-  private String[] createColumnArrayFromMeta() {
-    String[] metaColumnNames = new String[getInputRowMeta().size()];
-    for (int j = 0; j < getInputRowMeta().size(); j++) {
-      metaColumnNames[j] = getInputRowMeta().getValueMeta(j).getName();
-    }
-    return metaColumnNames;
-  }
-
-  private boolean[] createMeasureMappigs(List<String> measureCol) {
-    int size = getInputRowMeta().size();
-    boolean[] measurePresentMapping = new boolean[size];
-    for (int j = 0; j < size; j++) {
-      String columnName = getInputRowMeta().getValueMeta(j).getName();
-      for (String measure : measureCol) {
-        if (measure.equalsIgnoreCase(columnName)) {
-          measurePresentMapping[j] = true;
-          break;
-        }
-      }
-    }
-    return measurePresentMapping;
-
-  }
-
-  private boolean[] createMeasureSurrogateReqMapping() {
-    int size = getInputRowMeta().size();
-    boolean[] measureSuurogateReqMapping = new boolean[size];
-    for (int j = 0; j < size; j++) {
-      String columnName = getInputRowMeta().getValueMeta(j).getName();
-      Boolean isPresent = meta.getMeasureSurrogateRequired().get(columnName);
-      if (null != isPresent && isPresent) {
-        measureSuurogateReqMapping[j] = true;
-      }
-    }
-    return measureSuurogateReqMapping;
-  }
-
-  private void createForeignKeyMappingColumns() {
-    int size = getInputRowMeta().size();
-    foreignKeyMappingColumns = new String[size];
-    foreignKeyMappingColumnsForMultiple = new String[size][];
-    for (int j = 0; j < size; j++) {
-      String columnName = getInputRowMeta().getValueMeta(j).getName();
-      String foreignKeyColumnName = meta.foreignKeyPrimaryKeyMap.get(columnName);
-      if (foreignKeyColumnName != null) {
-        if (foreignKeyColumnName.indexOf(CarbonCommonConstants.COMA_SPC_CHARACTER) > -1) {
-          String[] splittedHiers =
-              foreignKeyColumnName.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-          foreignKeyMappingColumnsForMultiple[j] = splittedHiers;
-          foreignKeyMappingColumns[j] = foreignKeyColumnName;
-        } else {
-          foreignKeyMappingColumns[j] = foreignKeyColumnName;
-        }
-      }
-    }
-  }
-
-  private int createSurrogateForMeasure(String member, String columnName)
-      throws KettleException {
-    String colName = meta.getTableName() + '_' + columnName;
-    return data.getSurrogateKeyGen().getSurrogateForMeasure(member, colName);
-  }
-
-  private void insertHierarichies(Object[] rowWithKeys) throws KettleException {
-
-    try {
-      for (String hierName : denormHierarchies) {
-
-        String storeLocation = "";
-        String hierInprogName = hierName + CarbonCommonConstants.HIERARCHY_FILE_EXTENSION;
-        HierarchyValueWriterForCSV hierWriter = nrmlizedHierWriterMap.get(hierInprogName);
-        storeLocation = loadFolderLoc;
-        if (null == filemanager) {
-          filemanager = new FileManager();
-          filemanager.setName(storeLocation);
-        }
-        if (null == hierWriter) {
-          FileData fileData = new FileData(hierInprogName, storeLocation);
-          hierWriter = new HierarchyValueWriterForCSV(hierInprogName, storeLocation);
-          filemanager.add(fileData);
-          nrmlizedHierWriterMap.put(hierInprogName, hierWriter);
-        }
-
-        int[] levelsIndxs = meta.hirches.get(hierName);
-        int[] levelSKeys = new int[levelsIndxs.length];
-
-        if (meta.complexTypes.get(meta.hierColumnMap.get(hierName)[0]) == null) {
-          for (int i = 0; i < levelSKeys.length; i++) {
-            levelSKeys[i] = (Integer) rowWithKeys[levelsIndxs[i]];
-          }
-
-          if (levelSKeys.length > 1) {
-            data.getSurrogateKeyGen().checkNormalizedHierExists(levelSKeys, hierName, hierWriter);
-          }
-        }
-      }
-    } catch (Exception e) {
-      throw new KettleException(e.getMessage(), e);
-    }
-  }
-
-  private boolean isMeasureColumn(String msName, boolean compareWithTable) {
-    String msrNameTemp;
-    for (String msrName : meta.measureColumn) {
-      msrNameTemp = msrName;
-      if (compareWithTable) {
-        msrNameTemp = meta.getTableName() + '_' + msrNameTemp;
-      }
-      if (msrNameTemp.equalsIgnoreCase(msName)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private int[] getMeasureOriginalIndexes(String[] originalMsrCols) {
-    List<String> currMsrCol = new ArrayList<String>(10);
-    for (int i = 0; i < getInputRowMeta().size(); i++) {
-      String columnName = getInputRowMeta().getValueMeta(i).getName();
-      for (String measureCol : originalMsrCols) {
-        if (measureCol.equalsIgnoreCase(columnName)) {
-          currMsrCol.add(columnName);
-          break;
-        }
-      }
-    }
-    String[] currentMsrCols = currMsrCol.toArray(new String[currMsrCol.size()]);
-
-    int[] indexs = new int[currentMsrCols.length];
-
-    for (int i = 0; i < currentMsrCols.length; i++) {
-      for (int j = 0; j < originalMsrCols.length; j++) {
-        if (currentMsrCols[i].equalsIgnoreCase(originalMsrCols[j])) {
-          indexs[i] = j;
-          break;
-        }
-      }
-    }
-
-    return indexs;
-  }
-
-  private int[] getMemberMappingOriginalIndexes() {
-    int[] memIndexes = new int[meta.dimLens.length + meta.msrs.length];
-    Arrays.fill(memIndexes, -1);
-    String actualColumnName = null;
-    List<String> allColumnsNamesFromCSV = new ArrayList<String>(10);
-    for (int i = 0; i < getInputRowMeta().size(); i++) {
-      allColumnsNamesFromCSV.add(getInputRowMeta().getValueMeta(i).getName());
-    }
-
-    List<String> currentColName = new ArrayList<String>(meta.actualDimArray.length);
-    List<String> duplicateNames = new ArrayList<String>(10);
-    for (int i = 0; i < getInputRowMeta().size(); i++) {
-      String columnName = getInputRowMeta().getValueMeta(i).getName();
-      String hier = meta.foreignKeyHierarchyMap.get(columnName);
-
-      String uniqueName = meta.getTableName() + '_' + columnName;
-      if (null != hier) {
-
-        if (hier.indexOf(CarbonCommonConstants.COMA_SPC_CHARACTER) > -1) {
-          getCurrenColForMultiHier(currentColName, hier);
-        } else {
-          String tableName = meta.getHierDimTableMap().get(hier);
-
-          String[] columns = meta.hierColumnMap.get(hier);
-
-          if (null != columns) {
-            for (String column : columns) {
-              //currentColumnNames[k++] = column;
-              currentColName.add(tableName + '_' + column);
-            }
-          }
-        }
-
-        if (isMeasureColumn(columnName, false)) {
-          currentColName.add(uniqueName);
-        }
-
-      } else // then it can be direct column name if not foreign key.
-      {
-        if (!meta.isAggregate()) {
-          currentColName.add(uniqueName);
-          //add to duplicate column list if it is a repeated column. it is required since the
-          // member mapping is 1 to 1 mapping
-          //of csv columns and schema columns. so if schema columns are repeated then we have to
-          // handle it in special way.
-          checkAndAddDuplicateCols(duplicateNames, uniqueName);
-        } else {
-          actualColumnName = meta.columnAndTableNameColumnMapForAggMap.get(columnName);
-          if (actualColumnName != null) {
-            currentColName.add(meta.columnAndTableNameColumnMapForAggMap.get(columnName));
-          } else {
-            currentColName.add(uniqueName);
-          }
-        }
-      }
-    }
-    //Add the duplicate columns at the end so that it won't create any problem with current mapping.
-    currentColName.addAll(duplicateNames);
-    String[] currentColNamesArray = currentColName.toArray(new String[currentColName.size()]);
-
-    // We will use same array for dimensions and measures
-    // First create the mapping for dimensions.
-    int dimIndex = 0;
-    Map<String, Boolean> counterMap = new HashMap<String, Boolean>(16);
-    // Setting dimPresent value in CSV order as we need it later
-    dimPresentCsvOrder = new boolean[meta.dimPresent.length];
-    // var used to set measures value (in next loop)
-    int toAddInIndex = 0;
-    int tmpIndex = 0;
-    for (int i = 0; i < currentColNamesArray.length; i++) {
-      if (isMeasureColumn(currentColNamesArray[i], true) && isNotInDims(currentColNamesArray[i])) {
-        continue;
-      }
-      int n = 0;
-      for (int j = 0; j < meta.actualDimArray.length; j++) {
-
-        if (currentColNamesArray[i].equalsIgnoreCase(meta.dimColNames[j])) {
-
-          String mapKey = currentColNamesArray[i] + "__" + j;
-          if (null == counterMap.get(mapKey)) {
-            dimPresentCsvOrder[tmpIndex] = meta.dimPresent[j];//CHECKSTYLE:ON
-            tmpIndex++;
-            counterMap.put(mapKey, true);
-            if (!meta.dimPresent[j]) {
-              dimIndex++;
-              continue;
-            }
-            memIndexes[dimIndex++] = n;
-            // Added one more value to memIndexes, increase counter
-            toAddInIndex++;
-            break;
-          } else {
-            n++;
-            continue;
-          }
-        }
-        if (meta.dimPresent[j]) {
-          n++;
-        }
-      }
-    }
-
-    for (int actDimLen = 0; actDimLen < meta.actualDimArray.length; actDimLen++) {
-      boolean found = false;
-      for (int csvHeadLen = 0; csvHeadLen < currentColNamesArray.length; csvHeadLen++) {
-        if (meta.dimColNames[actDimLen].equalsIgnoreCase(currentColNamesArray[csvHeadLen])) {
-          found = true;
-          break;
-        }
-      }
-
-      if (!found) {
-        dimIndex++;
-        toAddInIndex++;
-      }
-    }
-
-    // Now create the mapping of measures
-    // There may be case when measure column is present in the CSV file
-    // but not present in the schema , in that case we need to skip that column while
-    // sending the output to next step.
-    // Or Measure can be in any ordinal in the csv
-
-    int k = 0;
-    Map<String, Boolean> existsMap = new HashMap<String, Boolean>(16);
-
-    for (int i = 0; i < currentColNamesArray.length; i++) {
-      k = calculateMeasureOriginalIndexes(memIndexes, currentColNamesArray, dimIndex, toAddInIndex,
-          k, existsMap, i);
-    }
-
-    return memIndexes;
-  }
-
-  private void getCurrenColForMultiHier(List<String> currentColName, String hier) {
-    String[] splittedHiers = hier.split(CarbonCommonConstants.COMA_SPC_CHARACTER);
-    for (String hierName : splittedHiers) {
-      String tableName = meta.getHierDimTableMap().get(hierName);
-
-      String[] cols = meta.hierColumnMap.get(hierName);
-      if (null != cols) {
-        for (String column : cols) {
-          currentColName.add(tableName + '_' + column);
-        }
-      }
-    }
-  }
-
-  private void checkAndAddDuplicateCols(List<String> duplicateNames, String uniqueName) {
-    boolean exists = false;
-    for (int i = 0; i < meta.dimColNames.length; i++) {
-      if (uniqueName.equals(meta.dimColNames[i])) {
-        if (exists) {
-          duplicateNames.add(uniqueName);
-        }
-        exists = true;
-      }
-    }
-  }
-
-  /**
-   * calculateMeasureOriginalIndexes
-   *
-   * @param memIndexes
-   * @param currentColNamesArray
-   * @param dimIndex
-   * @param toAddInIndex
-   * @param k
-   * @param existsMap
-   * @param i
-   * @return
-   */
-  public int calculateMeasureOriginalIndexes(int[] memIndexes, String[] currentColNamesArray,
-      int dimIndex, int toAddInIndex, int k, Map<String, Boolean> existsMap, int i) {
-    for (int j = 0; j < meta.measureColumn.length; j++) {
-      if (currentColNamesArray[i]
-          .equalsIgnoreCase(meta.getTableName() + '_' + meta.measureColumn[j])) {
-        if (existsMap.get(meta.measureColumn[j]) == null) {
-          memIndexes[k + dimIndex] = toAddInIndex + j;
-          k++;
-          existsMap.put(meta.measureColumn[j], true);
-          break;
-        }
-      }
-    }
-    return k;
-  }
-
-  private boolean isNotInDims(String columnName) {
-    for (String dimName : meta.dimColNames) {
-      if (dimName.equalsIgnoreCase(columnName)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private void closeConnections() throws KettleException {
-    try {
-      for (Entry<String, Connection> entry : cons.entrySet()) {
-        entry.getValue().close();
-      }
-      cons.clear();
-    } catch (Exception ex) {
-      throw new KettleException(ex.getMessage(), ex);
-    }
-  }
-
-  /**
-   * According to the hierarchies,generate the varLengthKeyGenerator
-   *
-   * @param keyGenerators
-   * @param hirches
-   * @param dimLens
-   */
-  private void updateHierarchyKeyGenerators(Map<String, KeyGenerator> keyGenerators,
-      Map<String, int[]> hirches, int[] dimLens, String[] dimCols) {
-    //
-    String timeHierNameVal = "";
-    if (meta.getCarbonTime() == null || "".equals(meta.getCarbonTime())) {
-      timeHierNameVal = "";
-    } else {
-      String[] hies = meta.getCarbonTime().split(":");
-      timeHierNameVal = hies[1];
-    }
-
-    // Set<Entry<String,int[]>> hierSet = hirches.entrySet();
-    Iterator<Entry<String, int[]>> itr = hirches.entrySet().iterator();
-
-    while (itr.hasNext()) {
-      Entry<String, int[]> hieEntry = itr.next();
-
-      int[] a = hieEntry.getValue();
-      int[] lens = new int[a.length];
-      String name = hieEntry.getKey();
-      //
-      if (name.equalsIgnoreCase(timeHierNameVal)) {
-        for (int i = 0; i < a.length; i++) { //CHECKSTYLE:OFF
-          lens[i] = dimLens[a[i]];
-        } //CHECKSTYLE:ON
-      } else {
-        String[] columns = meta.hierColumnMap.get(name);
-
-        if (meta.getComplexTypes().get(columns[0]) != null) {
-          continue;
-        }
-        boolean isNoDictionary = false;
-        for (int i = 0; i < a.length; i++) {
-          if (null != meta.noDictionaryCols && isDimensionNoDictionary(meta.noDictionaryCols,
-              columns[i])) {
-            isNoDictionary = true;
-            break;
-          }
-        }
-        //if no dictionary column then do not populate the dim lens
-        if (isNoDictionary) {
-          continue;
-        }
-        //
-        for (int i = 0; i < a.length; i++) {
-          int newIndex = -1;
-          for (int j = 0; j < dimCols.length; j++) {
-            //
-            if (checkDimensionColName(dimCols[j], columns[i])) {
-              newIndex = j;
-              break;
-            }
-          } //CHECKSTYLE:OFF
-          lens[i] = dimLens[newIndex];
-        } //CHECKSTYLE:ON
-      }
-      //
-      KeyGenerator generator = KeyGeneratorFactory.getKeyGenerator(lens);
-      keyGenerators.put(name, generator);
-
-    }
-
-    Iterator<Entry<String, GenericDataType>> complexMap =
-        meta.getComplexTypes().entrySet().iterator();
-    while (complexMap.hasNext()) {
-      Entry<String, GenericDataType> complexDataType = complexMap.next();
-      List<GenericDataType> primitiveTypes = new ArrayList<GenericDataType>();
-      complexDataType.getValue().getAllPrimitiveChildren(primitiveTypes);
-      for (GenericDataType eachPrimitive : primitiveTypes) {
-        KeyGenerator generator = KeyGeneratorFactory.getKeyGenerator(new int[] { -1 });
-        keyGenerators.put(eachPrimitive.getName(), generator);
-      }
-    }
-  }
-
-  private boolean checkDimensionColName(String dimColName, String hierName) {
-    String[] tables = meta.getDimTableArray();
-
-    for (String table : tables) {
-      String hierWithTableName = table + '_' + hierName;
-      if (hierWithTableName.equalsIgnoreCase(dimColName)) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  public boolean init(StepMetaInterface smi, StepDataInterface sdi) {
-    meta = (CarbonCSVBasedSeqGenMeta) smi;
-    data = (CarbonCSVBasedSeqGenData) sdi;
-    return super.init(smi, sdi);
-  }
-
-  public void dispose(StepMetaInterface smi, StepDataInterface sdi) {
-    /**
-     * Fortify Fix: FORWARD_NULL
-     * Changed to ||
-     * previously there was && but actully in case any one the object being null can through the
-     * nullpointer exception
-     *
-     */
-    if (null == smi || null == sdi) {
-      return;
-    }
-
-    meta = (CarbonCSVBasedSeqGenMeta) smi;
-    data = (CarbonCSVBasedSeqGenData) sdi;
-    CarbonCSVBasedDimSurrogateKeyGen surKeyGen = data.getSurrogateKeyGen();
-
-    try {
-      closeConnections();
-      if (null != surKeyGen) {
-        surKeyGen.setHierCache(null);
-        surKeyGen.setHierCacheReverse(null);
-        surKeyGen.setTimeDimCache(null);
-        surKeyGen.setMax(null);
-        surKeyGen.setTimDimMax(null);
-        surKeyGen.close();
-      }
-    } catch (Exception e) {
-      LOGGER.error(e);
-    } finally {
-      if (null != surKeyGen) {
-        clearDictionaryCache();
-        surKeyGen.setDictionaryCaches(null);
-      }
-    }
-    nrmlizedHierWriterMap = null;
-    data.clean();
-    super.dispose(smi, sdi);
-    meta = null;
-    data = null;
-  }
-
-  /**
-   * This method will clear the dictionary access count so that any unused
-   * column can be removed from the cache
-   */
-  private void clearDictionaryCache() {
-    Map<String, Dictionary> dictionaryCaches = surrogateKeyGen.getDictionaryCaches();
-    List<Dictionary> reverseDictionaries = new ArrayList<>(dictionaryCaches.values());
-    for (int i = 0; i < reverseDictionaries.size(); i++) {
-      Dictionary dictionary = reverseDictionaries.get(i);
-      dictionary.clear();
-    }
-  }
-
-  private String processnoDictionaryDim(int index, String dimensionValue, String dataType,
-      boolean isStringDataType, ByteBuffer[] out) {
-    if (!(isStringDataType)) {
-      if (null == DataTypeUtil
-          .normalizeIntAndLongValues(dimensionValue, DataTypeUtil.getDataType(dataType))) {
-        dimensionValue = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
-      }
-    }
-    ByteBuffer buffer = ByteBuffer
-        .wrap(dimensionValue.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-    buffer.rewind();
-    out[index] = buffer;
-    return dimensionValue;
-  }
-
-  /**
-   * @param NoDictionaryDims
-   * @param columnName
-   * @return true if the dimension is high cardinality.
-   */
-  private boolean isDimensionNoDictionary(String[] NoDictionaryDims, String columnName) {
-    for (String colName : NoDictionaryDims) {
-      if (colName
-          .equalsIgnoreCase(meta.getTableName() + CarbonCommonConstants.UNDERSCORE + columnName)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * 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
-     */
-    public void notifyFailed(Throwable exception) throws RuntimeException {
-      exec.shutdownNow();
-      LOGGER.error(exception);
-      throw new RuntimeException(exception);
-    }
-  }
-
-  /**
-   * This method will fill the carbon measures
-   *
-   * @param measures
-   */
-  private void populateCarbonMeasures(String[] measures) {
-    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(
-        meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName());
-    meta.carbonMeasures = new CarbonMeasure[measures.length];
-    msrDataType = new DataType[measures.length];
-    for (int i = 0; i < measures.length; i++) {
-      CarbonMeasure carbonMeasure = carbonTable.getMeasureByName(meta.getTableName(), measures[i]);
-      msrDataType[i] = carbonMeasure.getDataType();
-      if (DataType.DECIMAL == carbonMeasure.getDataType()) {
-        meta.carbonMeasures[i] = carbonMeasure;
-      }
-    }
-  }
-
-  private CarbonDimension[] populateNameToCarbonDimensionMap() {
-    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(
-        meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName());
-    List<CarbonDimension> dimensionsList = carbonTable.getDimensionByTableName(meta.getTableName());
-    CarbonDimension[] dimensionOrdinalToDimensionMapping =
-        new CarbonDimension[meta.getColumnSchemaDetailsWrapper().getColumnSchemaDetailsMap()
-            .size()];
-    List<CarbonDimension> dimListExcludingNoDictionaryColumn = dimensionsList;
-    if (null != meta.getNoDictionaryDims() && meta.getNoDictionaryDims().length() > 0) {
-      dimListExcludingNoDictionaryColumn =
-          new ArrayList<>(dimensionsList.size() - meta.noDictionaryCols.length);
-      for (CarbonDimension dimension : dimensionsList) {
-        // Here if dimension.getEncoder() lnly contains Encoding.INVERTED_INDEX, it
-        // means that NoDicColumn using InvertedIndex, so not put it into dic dims list.
-        if (!dimension.getEncoder().isEmpty() && !((1 == dimension.getEncoder().size()) &&
-            dimension.getEncoder().contains(Encoding.INVERTED_INDEX))) {
-          dimListExcludingNoDictionaryColumn.add(dimension);
-        }
-      }
-    }
-    for (int i = 0; i < dimListExcludingNoDictionaryColumn.size(); i++) {
-      CarbonDimension dimension = dimListExcludingNoDictionaryColumn.get(meta.memberMapping[i]);
-      if (dimension.isComplex()) {
-        populateComplexDimension(dimensionOrdinalToDimensionMapping, dimension);
-      } else {
-        dimensionOrdinalToDimensionMapping[meta.memberMapping[i]] = dimension;
-      }
-    }
-    return dimensionOrdinalToDimensionMapping;
-  }
-
-  private void populateComplexDimension(CarbonDimension[] dimensionOrdinalToDimensionMapping,
-      CarbonDimension dimension) {
-    List<CarbonDimension> listOfChildDimensions = dimension.getListOfChildDimensions();
-    for (CarbonDimension childDimension : listOfChildDimensions) {
-      if (childDimension.isComplex()) {
-        populateComplexDimension(dimensionOrdinalToDimensionMapping, childDimension);
-      } else {
-        dimensionOrdinalToDimensionMapping[childDimension.getOrdinal()] = childDimension;
-      }
-    }
-  }
-
-}
-


[08/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java
deleted file mode 100644
index cb6c432..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.csvreaderstep;
-
-import java.util.List;
-
-/**
- * Vo class which will holds all the properties required to read and parse the
- * csv file
- */
-public class UnivocityCsvParserVo {
-
-  /**
-   * delimiter of the records
-   */
-  private String delimiter;
-
-  /**
-   * file encoding
-   */
-  private String encoding;
-
-  /**
-   * is header present in the file
-   */
-  private boolean headerPresent;
-
-  /**
-   * enclosure
-   */
-  private String enclosure;
-
-  /**
-   * escape enclosure
-   */
-  private boolean escapeEnclosure;
-
-  /**
-   * number of columns
-   */
-  private int numberOfColumns;
-
-  /**
-   * block details list, which will have
-   * all the detail if the block
-   */
-  private List<BlockDetails> blockDetailsList;
-
-  /**
-   * escape character;
-   */
-  private String escapeCharacter;
-
-  /**
-   * quote character;
-   */
-  private String quoteCharacter;
-
-  /**
-   * comment character;
-   */
-  private String commentCharacter;
-
-  /**
-   * max number of columns configured by user to be parsed in a row
-   */
-  private int maxColumns;
-
-  /**
-   * @return the delimiter
-   */
-  public String getDelimiter() {
-    return delimiter;
-  }
-
-  /**
-   * @param delimiter the delimiter to set
-   */
-  public void setDelimiter(String delimiter) {
-    this.delimiter = delimiter;
-  }
-
-  /**
-   * @return the encoding
-   */
-  public String getEncoding() {
-    return encoding;
-  }
-
-  /**
-   * @param encoding the encoding to set
-   */
-  public void setEncoding(String encoding) {
-    this.encoding = encoding;
-  }
-
-  /**
-   * @return the headerPresent
-   */
-  public boolean isHeaderPresent() {
-    return headerPresent;
-  }
-
-  /**
-   * @param headerPresent the headerPresent to set
-   */
-  public void setHeaderPresent(boolean headerPresent) {
-    this.headerPresent = headerPresent;
-  }
-
-  /**
-   * @return the enclosure
-   */
-  public String getEnclosure() {
-    return enclosure;
-  }
-
-  /**
-   * @param enclosure the enclosure to set
-   */
-  public void setEnclosure(String enclosure) {
-    this.enclosure = enclosure;
-  }
-
-  /**
-   * @return the escapeEnclosure
-   */
-  public boolean isEscapeEnclosure() {
-    return escapeEnclosure;
-  }
-
-  /**
-   * @param escapeEnclosure the escapeEnclosure to set
-   */
-  public void setEscapeEnclosure(boolean escapeEnclosure) {
-    this.escapeEnclosure = escapeEnclosure;
-  }
-
-  /**
-   * @return the numberOfColumns
-   */
-  public int getNumberOfColumns() {
-    return numberOfColumns;
-  }
-
-  /**
-   * @param numberOfColumns the numberOfColumns to set
-   */
-  public void setNumberOfColumns(int numberOfColumns) {
-    this.numberOfColumns = numberOfColumns;
-  }
-
-  /**
-   * @return the blockDetailsList
-   */
-  public List<BlockDetails> getBlockDetailsList() {
-    return blockDetailsList;
-  }
-
-  /**
-   * @param blockDetailsList the blockDetailsList to set
-   */
-  public void setBlockDetailsList(List<BlockDetails> blockDetailsList) {
-    this.blockDetailsList = blockDetailsList;
-  }
-
-  /**
-   * @return the escapeCharacter
-   */
-  public String getEscapeCharacter() {
-    return escapeCharacter;
-  }
-
-  /**
-   * @param escapeCharacter the escapeCharacter to set
-   */
-  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;
-  }
-
-
-  /**
-   * @return
-   */
-  public int getMaxColumns() {
-    return maxColumns;
-  }
-
-  /**
-   * @param maxColumns
-   */
-  public void setMaxColumns(int maxColumns) {
-    this.maxColumns = maxColumns;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/step-attributes.xml
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/step-attributes.xml b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/step-attributes.xml
deleted file mode 100644
index 32b1452..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/step-attributes.xml
+++ /dev/null
@@ -1,229 +0,0 @@
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<attributes>
-
-  <!-- Top level attributes -->
-  <attribute id="FILENAME">
-    <xmlcode>filename</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.Filename.Label</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="FILENAME_FIELD">
-    <xmlcode>filename_field</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.FilenameField.Label</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="INCLUDE_FILENAME">
-    <xmlcode>include_filename</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.IncludeFilenameField.Label</description>
-    <tooltip/>
-    <valuetype>Boolean</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="ROW_NUM_FIELD">
-    <xmlcode>rownum_field</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.RowNumField.Label</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="HEADER_PRESENT">
-    <xmlcode>header</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.HeaderPresent.Label</description>
-    <tooltip/>
-    <valuetype>Boolean</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="DELIMITER">
-    <xmlcode>separator</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.Delimiter.Label</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="ENCLOSURE">
-    <xmlcode>enclosure</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.Enclosure.Label</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="ESACAPE_ENCLOSURE">
-    <xmlcode>escape_enclosure</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.EscapeEnclosure.Label</description>
-    <tooltip/>
-    <valuetype>Boolean</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="BUFFERSIZE">
-    <xmlcode>buffer_size</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.BufferSize.Label</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="LAZY_CONVERSION">
-    <xmlcode>lazy_conversion</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.LazyConversion.Label</description>
-    <tooltip/>
-    <valuetype>Boolean</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="PARALLEL">
-    <xmlcode>parallel</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.RunningInParallel.Label</description>
-    <tooltip/>
-    <valuetype>Boolean</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="NEWLINE_POSSIBLE">
-    <xmlcode>newline_possible</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.NewlinePossible.Label</description>
-    <tooltip/>
-    <valuetype>Boolean</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="ADD_FILENAME_RESULT">
-    <xmlcode>add_filename_result</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.AddResult.Label</description>
-    <tooltip>CsvInputDialog.AddResult.Tooltip</tooltip>
-    <valuetype>Boolean</valuetype>
-    <parentid/>
-  </attribute>
-  <attribute id="ENCODING">
-    <xmlcode>encoding</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.Encoding.Label</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid/>
-  </attribute>
-
-  <!-- The fields grid defining all CSV Input fields -->
-  <attribute id="FIELDS">
-    <xmlcode>fields</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.Fields.Label</description>
-    <tooltip/>
-    ValueMetaInterface.TYPE_NONE
-    <parentid/>
-  </attribute>
-
-  <!-- The definition of a single CSV input field -->
-  <attribute id="FIELD">
-    <xmlcode>field</xmlcode>
-    <repcode/>
-    <description>CsvInputDialog.Field.Label</description>
-    <tooltip/>
-    ValueMetaInterface.TYPE_NONE
-    <parentid>FIELDS</parentid>
-  </attribute>
-
-  <attribute id="FIELD_NAME">
-    <xmlcode>name</xmlcode>
-    <repcode>field_name</repcode>
-    <description>CsvInputDialog.NameColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-  <attribute id="FIELD_TYPE">
-    <xmlcode>type</xmlcode>
-    <repcode>field_type</repcode>
-    <description>CsvInputDialog.TypeColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-  <attribute id="FIELD_FORMAT">
-    <xmlcode>format</xmlcode>
-    <repcode>field_format</repcode>
-    <description>CsvInputDialog.FormatColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-  <attribute id="FIELD_LENGTH">
-    <xmlcode>length</xmlcode>
-    <repcode>field_length</repcode>
-    <description>CsvInputDialog.LengthColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-  <attribute id="FIELD_PRECISION">
-    <xmlcode>precision</xmlcode>
-    <repcode>field_precision</repcode>
-    <description>CsvInputDialog.PrecisionColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-  <attribute id="FIELD_CURRENCY">
-    <xmlcode>currency</xmlcode>
-    <repcode>field_currency</repcode>
-    <description>CsvInputDialog.CurrencyColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-  <attribute id="FIELD_DECIMAL">
-    <xmlcode>mantissa</xmlcode>
-    <repcode>field_decimal</repcode>
-    <description>CsvInputDialog.DecimalColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-  <attribute id="FIELD_GROUP">
-    <xmlcode>group</xmlcode>
-    <repcode>field_group</repcode>
-    <description>CsvInputDialog.GroupColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-  <attribute id="FIELD_TRIM_TYPE">
-    <xmlcode>trim_type</xmlcode>
-    <repcode>field_trim_type</repcode>
-    <description>CsvInputDialog.TrimTypeColumn.Column</description>
-    <tooltip/>
-    <valuetype>String</valuetype>
-    <parentid>FIELD</parentid>
-  </attribute>
-
-</attributes>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
deleted file mode 100644
index eba739a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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;
-
-import java.io.Serializable;
-import java.sql.Timestamp;
-import java.util.List;
-
-public class DataProcessTaskStatus implements IDataProcessStatus, Serializable {
-
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * DataLoader Status Identifier.
-   */
-  private int dataloadstatusid;
-
-  /**
-   *
-   */
-  private Timestamp createdTime;
-
-  /**
-   * Status Identifier.
-   */
-  private String key;
-
-  /**
-   * Status .
-   */
-  private String status;
-
-  /**
-   * description for the task
-   */
-  private String desc;
-
-  /**
-   * task type
-   */
-  private int taskType;
-
-  private String databaseName;
-
-  private String tableName;
-
-  private String newSchemaFilePath;
-
-  private String oldSchemaFilePath;
-
-  private String csvFilePath;
-
-  private String dimTables;
-
-  private boolean isDirectLoad;
-  private List<String> filesToProcess;
-  private String csvHeader;
-  private String csvDelimiter;
-  /**
-   * Set if the call to restructre from path or by upload
-   */
-  private boolean isFromPathApi;
-
-  private String blocksID;
-
-  private String escapeCharacter;
-
-  private String quoteCharacter;
-
-  private String commentCharacter;
-
-  private String rddIteratorKey;
-
-  private String dateFormat;
-
-  public DataProcessTaskStatus(String databaseName, String tableName) {
-    this.databaseName = databaseName;
-    this.tableName = tableName;
-    this.desc = "";
-    this.setNewSchemaFilePath("");
-    this.setOldSchemaFilePath("");
-  }
-
-  public DataProcessTaskStatus() {
-  }
-
-  public boolean isDirectLoad() {
-    return isDirectLoad;
-  }
-
-  public void setDirectLoad(boolean isDirectLoad) {
-    this.isDirectLoad = isDirectLoad;
-  }
-
-  public List<String> getFilesToProcess() {
-    return filesToProcess;
-  }
-
-  public void setFilesToProcess(List<String> filesToProcess) {
-    this.filesToProcess = filesToProcess;
-  }
-
-  public String getCsvHeader() {
-    return csvHeader;
-  }
-
-  public void setCsvHeader(String csvHeader) {
-    this.csvHeader = csvHeader;
-  }
-
-  public String getCsvDelimiter() {
-    return csvDelimiter;
-  }
-
-  public void setCsvDelimiter(String csvDelimiter) {
-    this.csvDelimiter = csvDelimiter;
-  }
-
-  /**
-   * @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;
-  }
-
-  public String getDesc() {
-    return desc;
-  }
-
-  public void setDesc(String desc) {
-    this.desc = desc;
-  }
-
-  @Override public String getKey() {
-    return key;
-  }
-
-  public void setKey(String key) {
-    this.key = key;
-  }
-
-  @Override public String getStatus() {
-    return status;
-  }
-
-  public void setStatus(String status) {
-    this.status = status;
-  }
-
-  public int getDataloadstatusid() {
-    return dataloadstatusid;
-  }
-
-  public void setDataloadstatusid(int dataloadstatusid) {
-    this.dataloadstatusid = dataloadstatusid;
-  }
-
-  public Timestamp getCreatedTime() {
-    return createdTime;
-  }
-
-  public void setCreatedTime(Timestamp createdTime) {
-    this.createdTime = createdTime;
-  }
-
-  public int getTaskType() {
-    return taskType;
-  }
-
-  public void setTaskType(int taskType) {
-    this.taskType = taskType;
-  }
-
-  public String getOldSchemaFilePath() {
-    return oldSchemaFilePath;
-  }
-
-  public void setOldSchemaFilePath(String oldSchemaFilePath) {
-    this.oldSchemaFilePath = oldSchemaFilePath;
-  }
-
-  public String getNewSchemaFilePath() {
-    return newSchemaFilePath;
-  }
-
-  public void setNewSchemaFilePath(String newSchemaFilePath) {
-    this.newSchemaFilePath = newSchemaFilePath;
-  }
-
-  public String getCsvFilePath() {
-    return csvFilePath;
-  }
-
-  public void setCsvFilePath(String csvFilePath) {
-    this.csvFilePath = csvFilePath;
-  }
-
-  public String getDimTables() {
-    return dimTables;
-  }
-
-  public void setDimTables(String dimTables) {
-    this.dimTables = dimTables;
-  }
-
-  public boolean isFromPathApi() {
-    return isFromPathApi;
-  }
-
-  public void setFromPathApi(boolean isFromPathApi) {
-    this.isFromPathApi = isFromPathApi;
-  }
-
-  /**
-   * to make a copy
-   */
-  public IDataProcessStatus makeCopy() {
-    IDataProcessStatus copy = new DataProcessTaskStatus();
-    copy.setTableName(this.tableName);
-    copy.setDataloadstatusid(this.dataloadstatusid);
-    copy.setDesc(this.desc);
-    copy.setKey(this.key);
-    copy.setDatabaseName(databaseName);
-    copy.setStatus(status);
-    return copy;
-  }
-
-  public String getBlocksID() {
-    return blocksID;
-  }
-
-  public void setBlocksID(String blocksID) {
-    this.blocksID = blocksID;
-  }
-
-  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 getRddIteratorKey() {
-    return rddIteratorKey;
-  }
-
-  public void setRddIteratorKey(String rddIteratorKey) {
-    this.rddIteratorKey = rddIteratorKey;
-  }
-
-  public String getDateFormat() { return dateFormat; }
-
-  public void setDateFormat(String dateFormat) { this.dateFormat = dateFormat; }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java
deleted file mode 100644
index 412c1ff..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.dataprocessor;
-
-import java.sql.Timestamp;
-import java.util.List;
-
-public interface IDataProcessStatus {
-
-  /**
-   * serial id
-   *
-   * @return
-   */
-  int getDataloadstatusid();
-
-  /**
-   * @param dataloadstatusid
-   */
-  void setDataloadstatusid(int dataloadstatusid);
-
-  /**
-   * @return the createdTime
-   */
-  Timestamp getCreatedTime();
-
-  /**
-   * @param createdTime the createdTime to set
-   */
-  void setCreatedTime(Timestamp createdTime);
-
-  /**
-   * return the description of the task
-   */
-  String getDesc();
-
-  /**
-   * set the description of the task
-   */
-  void setDesc(String desc);
-
-  /**
-   * This method is used to get the Key for saving status of data loading.
-   *
-   * @return String - Key (databaseName + tableName + tableName).
-   */
-  String getKey();
-
-  /**
-   * @param key
-   */
-  void setKey(String key);
-
-  /**
-   * To get the status of the data loading.
-   *
-   * @return String - Status
-   */
-  String getStatus();
-
-  /**
-   * To set the status of the data loading.
-   */
-  void setStatus(String status);
-
-  /**
-   * Return task type
-   * 1- DATALOADING 2- RESTRUCTURE
-   */
-  int getTaskType();
-
-  /**
-   * 1- DATALOADING 2- RESTRUCTURE
-   */
-  void setTaskType(int taskType);
-
-  /**
-   * @return the databaseName
-   */
-  String getDatabaseName();
-
-  /**
-   * @param databaseName the databaseName to set
-   */
-  void setDatabaseName(String databaseName);
-
-  /**
-   * @return the tableName
-   */
-  String getTableName();
-
-  /**
-   * @param tableName the tableName to set
-   */
-  void setTableName(String tableName);
-
-  /**
-   * @return the oldSchemaFilePath
-   */
-  String getOldSchemaFilePath();
-
-  /**
-   * @param oldSchemaFilePath the oldSchemaFilePath to set
-   */
-  void setOldSchemaFilePath(String oldSchemaFilePath);
-
-  /**
-   * @return the newSchemaFilePath
-   */
-  String getNewSchemaFilePath();
-
-  /**
-   * @param newSchemaFilePath the newSchemaFilePath to set
-   */
-  void setNewSchemaFilePath(String newSchemaFilePath);
-
-  /**
-   * @return the csvFilePath
-   */
-  String getCsvFilePath();
-
-  /**
-   * @param csvFilePath the csvFilePath to set
-   */
-  void setCsvFilePath(String csvFilePath);
-
-  /**
-   * @return the dimTables
-   */
-  String getDimTables();
-
-  /**
-   * @param dimTables the dimTables to set
-   */
-  void setDimTables(String dimTables);
-
-  /**
-   * @return the isFromPathApi
-   */
-  boolean isFromPathApi();
-
-  /**
-   * @param isFromPathApi the isFromPathApi to set
-   */
-  void setFromPathApi(boolean isFromPathApi);
-
-  /**
-   * @return
-   */
-  IDataProcessStatus makeCopy();
-
-  boolean isDirectLoad();
-
-  void setDirectLoad(boolean isDirectLoad);
-
-  List<String> getFilesToProcess();
-
-  void setFilesToProcess(List<String> filesToProcess);
-
-  String getCsvHeader();
-
-  void setCsvHeader(String csvHeader);
-
-  String getCsvDelimiter();
-
-  void setCsvDelimiter(String csvDelimiter);
-
-  String getBlocksID();
-
-  String getEscapeCharacter();
-
-  String getQuoteCharacter();
-
-  String getCommentCharacter();
-
-  String getRddIteratorKey();
-
-  String getDateFormat();
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/RecordComparator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/RecordComparator.java b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/RecordComparator.java
deleted file mode 100644
index ea7945f..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/queue/impl/RecordComparator.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.dataprocessor.queue.impl;
-
-import java.util.Comparator;
-
-import org.apache.carbondata.processing.dataprocessor.record.holder.DataProcessorRecordHolder;
-
-public class RecordComparator implements Comparator<DataProcessorRecordHolder> {
-
-  @Override public int compare(DataProcessorRecordHolder o1, DataProcessorRecordHolder o2) {
-
-    if (o1 == null) {
-      return -1;
-    }
-    if (o2 == null) {
-      return 1;
-    }
-
-    if (o1.getSeqNumber() < o2.getSeqNumber()) {
-      return -1;
-    } else if (o1.getSeqNumber() > o2.getSeqNumber()) {
-      return 1;
-    }
-    return 0;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index 68ca84c..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/record/holder/DataProcessorRecordHolder.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.dataprocessor.record.holder;
-
-/**
- * Data processor for the record.
- */
-public class DataProcessorRecordHolder {
-
-  private int seqNumber;
-
-  public DataProcessorRecordHolder(int size, int seqNumber) {
-    this.seqNumber = seqNumber;
-  }
-
-  /**
-   * Returns the sequence number.
-   */
-  public int getSeqNumber() {
-    return seqNumber;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 442c0f0..02ceb06 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
@@ -23,14 +23,11 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 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.surrogatekeysgenerator.csvbased.CarbonCSVBasedDimSurrogateKeyGen;
 
-import org.pentaho.di.core.exception.KettleException;
 
 /**
  * Array DataType stateless object used in data loading
@@ -143,32 +140,6 @@ public class ArrayDataType implements GenericDataType<ArrayObject> {
 
   }
 
-  /*
-   * parse string and generate surrogate
-   */
-  @Override
-  public void parseStringAndWriteByteArray(String tableName, String inputString,
-      String[] delimiter, int delimiterIndex, DataOutputStream dataOutputStream,
-      CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) throws KettleException, IOException {
-
-    if (inputString == null || "null".equals(inputString) || "".equals(inputString) ||
-        CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(inputString)) {
-      dataOutputStream.writeInt(1);
-      children.parseStringAndWriteByteArray(tableName,
-          CarbonCommonConstants.MEMBER_DEFAULT_VAL, delimiter, delimiterIndex, dataOutputStream,
-          surrogateKeyGen);
-    } else {
-      String[] splitInput = inputString.split(delimiter[delimiterIndex], -1);
-      dataOutputStream.writeInt(splitInput.length);
-      delimiterIndex =
-          (delimiter.length - 1) == delimiterIndex ? delimiterIndex : delimiterIndex + 1;
-      for (String eachInput : splitInput) {
-        children.parseStringAndWriteByteArray(tableName, eachInput, delimiter, delimiterIndex,
-            dataOutputStream, surrogateKeyGen);
-      }
-    }
-  }
-
   @Override
   public void writeByteArray(ArrayObject input, DataOutputStream dataOutputStream)
       throws IOException, DictionaryGenerationException {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 91caf7a..6b54d2d 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
@@ -26,9 +26,6 @@ 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.surrogatekeysgenerator.csvbased.CarbonCSVBasedDimSurrogateKeyGen;
-
-import org.pentaho.di.core.exception.KettleException;
 
 /**
  * Generic DataType interface which will be used while data loading for complex types like Array &
@@ -57,21 +54,6 @@ public interface GenericDataType<T> {
   void getAllPrimitiveChildren(List<GenericDataType> primitiveChild);
 
   /**
-   * Split raw csv data into seperate column using delimiter and generate surrogate key
-   * @param tableName
-   * @param inputString
-   * @param delimiter
-   * @param delimiterIndex
-   * @param dataOutputStream
-   * @param surrogateKeyGen
-   * @throws KettleException
-   * @throws IOException
-   */
-  void parseStringAndWriteByteArray(String tableName, String inputString, String[] delimiter,
-      int delimiterIndex, DataOutputStream dataOutputStream,
-      CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) throws KettleException, IOException;
-
-  /**
    * writes to byte stream
    * @param dataOutputStream
    * @throws IOException

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 105f5f4..e7e48e9 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,6 @@ import org.apache.carbondata.core.util.DataTypeUtil;
 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.surrogatekeysgenerator.csvbased.CarbonCSVBasedDimSurrogateKeyGen;
-
-import org.pentaho.di.core.exception.KettleException;
 
 /**
  * Primitive DataType stateless object used in data loading
@@ -218,28 +215,6 @@ public class PrimitiveDataType implements GenericDataType<Object> {
     index = surrIndex;
   }
 
-  /*
-   * parse string and generate surrogate
-   */
-  @Override public void parseStringAndWriteByteArray(String tableName, String inputString,
-      String[] delimiter, int delimiterIndex, DataOutputStream dataOutputStream,
-      CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) throws KettleException, IOException {
-    String parsedValue = DataTypeUtil.parseValue(inputString,
-        surrogateKeyGen.getDimensionOrdinalToDimensionMapping()[dimensionOrdinal]);
-    Integer surrogateKey = null;
-    if (null == parsedValue) {
-      surrogateKey = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY;
-    } else {
-      surrogateKey = surrogateKeyGen
-          .generateSurrogateKeys(parsedValue, tableName + CarbonCommonConstants.UNDERSCORE + name,
-              this.getColumnId());
-      if (surrogateKey == CarbonCommonConstants.INVALID_SURROGATE_KEY) {
-        surrogateKey = CarbonCommonConstants.MEMBER_DEFAULT_VAL_SURROGATE_KEY;
-      }
-    }
-    dataOutputStream.writeInt(surrogateKey);
-  }
-
   @Override public void writeByteArray(Object input, DataOutputStream dataOutputStream)
       throws IOException, DictionaryGenerationException {
     String parsedValue =

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 a131004..a61144e 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
@@ -23,14 +23,10 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 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.surrogatekeysgenerator.csvbased.CarbonCSVBasedDimSurrogateKeyGen;
-
-import org.pentaho.di.core.exception.KettleException;
 
 /**
  * Struct DataType stateless object used in data loading
@@ -144,40 +140,6 @@ public class StructDataType implements GenericDataType<StructObject> {
 
   }
 
-  /*
-   * parse string and generate surrogate
-   */
-  @Override
-  public void parseStringAndWriteByteArray(String tableName, String inputString,
-      String[] delimiter, int delimiterIndex, DataOutputStream dataOutputStream,
-      CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) throws KettleException, IOException {
-    if (inputString == null || "null".equals(inputString)) {
-      // Indicates null array
-      dataOutputStream.writeInt(children.size());
-      // For other children elements which dont have data, write empty
-      for (int i = 0; i < children.size(); i++) {
-        children.get(i).parseStringAndWriteByteArray(tableName,
-            CarbonCommonConstants.MEMBER_DEFAULT_VAL, delimiter, delimiterIndex, dataOutputStream,
-            surrogateKeyGen);
-      }
-    } else {
-      String[] splitInput = inputString.split(delimiter[delimiterIndex], -1);
-      dataOutputStream.writeInt(children.size());
-      delimiterIndex =
-          (delimiter.length - 1) == delimiterIndex ? delimiterIndex : delimiterIndex + 1;
-      for (int i = 0; i < splitInput.length && i < children.size(); i++) {
-        children.get(i).parseStringAndWriteByteArray(tableName, splitInput[i], delimiter,
-            delimiterIndex, dataOutputStream, surrogateKeyGen);
-      }
-      // For other children elements which dont have data, write empty
-      for (int i = splitInput.length; i < children.size(); i++) {
-        children.get(i).parseStringAndWriteByteArray(tableName,
-            CarbonCommonConstants.MEMBER_DEFAULT_VAL, delimiter, delimiterIndex, dataOutputStream,
-            surrogateKeyGen);
-      }
-    }
-  }
-
   @Override public void writeByteArray(StructObject input, DataOutputStream dataOutputStream)
       throws IOException, DictionaryGenerationException {
     dataOutputStream.writeInt(children.size());

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/exception/CarbonDataProcessorException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/exception/CarbonDataProcessorException.java b/processing/src/main/java/org/apache/carbondata/processing/exception/CarbonDataProcessorException.java
deleted file mode 100644
index 7552e6a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/exception/CarbonDataProcessorException.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.exception;
-
-import java.util.Locale;
-
-public class CarbonDataProcessorException 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 CarbonDataProcessorException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public CarbonDataProcessorException(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/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index 2876eb8..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
+++ /dev/null
@@ -1,965 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.graphgenerator;
-
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashSet;
-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.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-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.util.CarbonProperties;
-import org.apache.carbondata.processing.api.dataloader.DataLoadModel;
-import org.apache.carbondata.processing.api.dataloader.SchemaInfo;
-import org.apache.carbondata.processing.csvreaderstep.BlockDetails;
-import org.apache.carbondata.processing.csvreaderstep.CsvInputMeta;
-import org.apache.carbondata.processing.graphgenerator.configuration.GraphConfigurationInfo;
-import org.apache.carbondata.processing.mdkeygen.MDKeyGenStepMeta;
-import org.apache.carbondata.processing.merger.step.CarbonSliceMergerStepMeta;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.schema.metadata.TableOptionWrapper;
-import org.apache.carbondata.processing.sortandgroupby.sortdatastep.SortKeyStepMeta;
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.CarbonCSVBasedSeqGenMeta;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-import org.apache.carbondata.processing.util.CarbonSchemaParser;
-import org.apache.carbondata.processing.util.NonDictionaryUtil;
-
-import org.pentaho.di.core.KettleEnvironment;
-import org.pentaho.di.core.database.DatabaseMeta;
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.util.EnvUtil;
-import org.pentaho.di.trans.TransHopMeta;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-import org.pentaho.di.trans.steps.hadoopfileinput.HadoopFileInputMeta;
-import org.pentaho.di.trans.steps.selectvalues.SelectMetadataChange;
-import org.pentaho.di.trans.steps.selectvalues.SelectValuesMeta;
-import org.pentaho.di.trans.steps.tableinput.TableInputMeta;
-
-public class GraphGenerator {
-
-  public static final HashMap<String, BlockDetails[]> blockInfo = new HashMap<>();
-  /**
-   * DEFAUL_BLOCKLET_SIZE
-   */
-  private static final String DEFAUL_BLOCKLET_SIZE = "8192";
-  /**
-   * DEFAULE_MAX_BLOCKLET_IN_FILE
-   */
-  private static final String DEFAULE_MAX_BLOCKLET_IN_FILE = "100";
-  /**
-   * DEFAULT_NUMBER_CORES
-   */
-  private static final String DEFAULT_NUMBER_CORES = "2";
-  /**
-   * DEFAULT_BATCH_SIZE
-   */
-  private static final String DEFAULT_BATCH_SIZE = "1000";
-  /**
-   * DEFAULT_SORT_SIZE
-   */
-  private static final String DEFAULT_SORT_SIZE = "100000";
-  /**
-   * drivers
-   */
-  private static final Map<String, String> DRIVERS;
-  /**
-   * Comment for <code>LOGGER</code>
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(GraphGenerator.class.getName());
-  /**
-   * kettleInitialized
-   */
-  private static boolean kettleInitialized = false;
-
-  static {
-
-    DRIVERS = new HashMap<String, String>(1);
-
-    DRIVERS.put("oracle.jdbc.OracleDriver", CarbonCommonConstants.TYPE_ORACLE);
-    DRIVERS.put("com.mysql.jdbc.Driver", CarbonCommonConstants.TYPE_MYSQL);
-    DRIVERS.put("org.gjt.mm.mysql.Driver", CarbonCommonConstants.TYPE_MYSQL);
-    DRIVERS.put("com.microsoft.sqlserver.jdbc.SQLServerDriver", CarbonCommonConstants.TYPE_MSSQL);
-    DRIVERS.put("com.sybase.jdbc3.jdbc.SybDriver", CarbonCommonConstants.TYPE_SYBASE);
-  }
-
-  /**
-   * OUTPUT_LOCATION
-   */
-  private String outputLocation = "";
-  /**
-   * xAxixLocation
-   */
-  private int xAxixLocation = 50;
-  /**
-   * yAxixLocation
-   */
-  private int yAxixLocation = 100;
-  /**
-   * databaseName
-   */
-  private String databaseName;
-  /**
-   * table
-   */
-  //    private Table table;
-  /**
-   * instance
-   */
-  private CarbonProperties instance;
-  /**
-   * schemaInfo
-   */
-  private SchemaInfo schemaInfo;
-  /**
-   * Table name
-   */
-  private String tableName;
-  /**
-   * Is CSV Load request
-   */
-  private boolean isCSVLoad;
-  /**
-   * Modified dimension
-   */
-  private String[] modifiedDimension;
-  /**
-   * isAutoAggRequest
-   */
-  private boolean isAutoAggRequest;
-  /**
-   * schema
-   */
-  private CarbonDataLoadSchema carbonDataLoadSchema;
-  /**
-   * isUpdateMemberRequest
-   */
-  private boolean isUpdateMemberRequest;
-  /**
-   * If the CSV file is present in HDFS?
-   */
-  private boolean isHDFSReadMode;
-  /**
-   * partitionID
-   */
-  private String partitionID;
-  private boolean isColumnar;
-  private String factTableName;
-  private String factStoreLocation;
-  private String blocksID;
-  private String escapeCharacter;
-  private String quoteCharacter;
-  private String commentCharacter;
-  private String dateFormat;
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-  /**
-   * load Id
-   */
-  private String segmentId;
-  /**
-   * new load start time
-   */
-  private String factTimeStamp;
-  /**
-   * max number of columns configured by user to be parsed in a row
-   */
-  private String maxColumns;
-
-  private String rddIteratorKey;
-
-  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.isAutoAggRequest = schemaInfo.isAutoAggregateRequest();
-    this.carbonDataLoadSchema = carbonDataLoadSchema;
-    this.databaseName = carbonDataLoadSchema.getCarbonTable().getDatabaseName();
-    this.partitionID = partitionID;
-    this.factStoreLocation = factStoreLocation;
-    this.isColumnar = Boolean.parseBoolean(CarbonCommonConstants.IS_COLUMNAR_STORAGE_DEFAULTVALUE);
-    this.blocksID = dataLoadModel.getBlocksID();
-    this.taskNo = dataLoadModel.getTaskNo();
-    this.quoteCharacter = dataLoadModel.getQuoteCharacter();
-    this.commentCharacter = dataLoadModel.getCommentCharacter();
-    this.dateFormat = dataLoadModel.getDateFormat();
-    this.factTimeStamp = dataLoadModel.getFactTimeStamp();
-    this.segmentId = segmentId;
-    this.escapeCharacter = dataLoadModel.getEscapeCharacter();
-    this.maxColumns = dataLoadModel.getMaxColumns();
-    initialise();
-    LOGGER.info("************* Is Columnar Storage" + isColumnar);
-  }
-
-  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();
-  }
-
-  /**
-   * Generate the graph file ...
-   *
-   * @param transMeta
-   * @param graphFile
-   * @throws KettleException
-   */
-  private static void generateGraphFile(TransMeta transMeta, String graphFile)
-      throws GraphGeneratorException {
-    //
-    DataOutputStream dos = null;
-    try {
-      String xml = transMeta.getXML();
-      dos = new DataOutputStream(new FileOutputStream(new File(graphFile)));
-      dos.write(xml.getBytes(CarbonCommonConstants.DEFAULT_CHARSET));
-    } catch (KettleException kettelException) {
-      throw new GraphGeneratorException("Error while getting the graph XML", kettelException);
-    }
-    //
-    catch (FileNotFoundException e) {
-      throw new GraphGeneratorException("Unable to find the graph fileL", e);
-    }
-    //
-    catch (UnsupportedEncodingException ue) {
-      throw new GraphGeneratorException("Error while Converting the graph xml string to bytes", ue);
-    }
-    //
-    catch (IOException ioe) {
-      throw new GraphGeneratorException("Error while writing the graph file", ioe);
-    } finally {
-      //
-      if (dos != null) {
-        try {
-          dos.close();
-        } catch (IOException e) {
-          e.getMessage();
-        }
-      }
-    }
-  }
-
-  private void initialise() {
-    this.instance = CarbonProperties.getInstance();
-    //TO-DO need to take care while supporting aggregate table using new schema.
-    //aggregateTable = CarbonSchemaParser.getAggregateTable(table, schema);
-    this.factTableName = carbonDataLoadSchema.getCarbonTable().getFactTableName();
-  }
-
-  public void generateGraph() throws GraphGeneratorException {
-    validateAndInitialiseKettelEngine();
-    GraphConfigurationInfo graphConfigInfoForFact = getGraphConfigInfoForFact(carbonDataLoadSchema);
-    generateGraph(graphConfigInfoForFact, graphConfigInfoForFact.getTableName() + ": Graph",
-        isCSVLoad, graphConfigInfoForFact);
-  }
-
-  private void validateAndInitialiseKettelEngine() throws GraphGeneratorException {
-    File file = new File(
-        outputLocation + File.separator + schemaInfo.getDatabaseName() + File.separator
-            + this.tableName + File.separator + this.segmentId + File.separator + this.taskNo
-            + File.separator);
-    boolean isDirCreated = false;
-    if (!file.exists()) {
-      isDirCreated = file.mkdirs();
-
-      if (!isDirCreated) {
-        LOGGER.error(
-            "Unable to create directory or directory already exist" + file.getAbsolutePath());
-        throw new GraphGeneratorException("INTERNAL_SYSTEM_ERROR");
-      }
-    }
-
-    synchronized (DRIVERS) {
-      try {
-        if (!kettleInitialized) {
-          EnvUtil.environmentInit();
-          KettleEnvironment.init();
-          kettleInitialized = true;
-        }
-      } catch (KettleException kettlExp) {
-        LOGGER.error(kettlExp);
-        throw new GraphGeneratorException("Error While Initializing the Kettel Engine ", kettlExp);
-      }
-    }
-  }
-
-  private void generateGraph(GraphConfigurationInfo configurationInfo, String transName,
-      boolean isCSV, GraphConfigurationInfo configurationInfoForFact)
-      throws GraphGeneratorException {
-
-    TransMeta trans = new TransMeta();
-    trans.setName(transName);
-
-    if (!isCSV) {
-      trans.addDatabase(getDatabaseMeta(configurationInfo));
-    }
-
-    trans.setSizeRowset(Integer.parseInt(instance
-        .getProperty(CarbonCommonConstants.GRAPH_ROWSET_SIZE,
-            CarbonCommonConstants.GRAPH_ROWSET_SIZE_DEFAULT)));
-
-    StepMeta inputStep = null;
-    StepMeta carbonSurrogateKeyStep = null;
-    StepMeta selectValueToChangeTheDataType = null;
-
-    // get all step
-    if (isCSV) {
-      if (isHDFSReadMode) {
-        inputStep = getHadoopInputStep(configurationInfo);
-      } else {
-        inputStep = getCSVInputStep(configurationInfo);
-      }
-    } else {
-      inputStep = getTableInputStep(configurationInfo);
-      selectValueToChangeTheDataType = getSelectValueToChangeTheDataType(configurationInfo, 1);
-    }
-    carbonSurrogateKeyStep = getCarbonCSVBasedSurrogateKeyStep(configurationInfo);
-    StepMeta sortStep = getSortStep(configurationInfo);
-    StepMeta carbonMDKeyStep = getMDKeyStep(configurationInfo);
-    StepMeta carbonSliceMergerStep = null;
-    carbonSliceMergerStep = getSliceMeregerStep(configurationInfo, configurationInfoForFact);
-
-    // add all steps to trans
-    trans.addStep(inputStep);
-
-    if (!isCSV) {
-      trans.addStep(selectValueToChangeTheDataType);
-    }
-
-    trans.addStep(carbonSurrogateKeyStep);
-    trans.addStep(sortStep);
-    trans.addStep(carbonMDKeyStep);
-
-    trans.addStep(carbonSliceMergerStep);
-    TransHopMeta inputStepToSelectValueHop = null;
-    TransHopMeta tableInputToSelectValue = null;
-
-    if (isCSV) {
-      inputStepToSelectValueHop = new TransHopMeta(inputStep, carbonSurrogateKeyStep);
-    } else {
-      inputStepToSelectValueHop = new TransHopMeta(inputStep, selectValueToChangeTheDataType);
-      tableInputToSelectValue =
-          new TransHopMeta(selectValueToChangeTheDataType, carbonSurrogateKeyStep);
-    }
-
-    // create hop
-    TransHopMeta surrogateKeyToSortHop = new TransHopMeta(carbonSurrogateKeyStep, sortStep);
-    TransHopMeta sortToMDKeyHop = new TransHopMeta(sortStep, carbonMDKeyStep);
-    TransHopMeta mdkeyToSliceMerger = null;
-    mdkeyToSliceMerger = new TransHopMeta(carbonMDKeyStep, carbonSliceMergerStep);
-
-    if (isCSV) {
-      trans.addTransHop(inputStepToSelectValueHop);
-    } else {
-      trans.addTransHop(inputStepToSelectValueHop);
-      trans.addTransHop(tableInputToSelectValue);
-    }
-
-    trans.addTransHop(surrogateKeyToSortHop);
-    trans.addTransHop(sortToMDKeyHop);
-    trans.addTransHop(mdkeyToSliceMerger);
-
-    String graphFilePath =
-        outputLocation + File.separator + schemaInfo.getDatabaseName() + File.separator
-            + this.tableName + File.separator + segmentId + File.separator + this.taskNo
-            + File.separator + this.tableName + ".ktr";
-    generateGraphFile(trans, graphFilePath);
-  }
-
-  private StepMeta getHadoopInputStep(GraphConfigurationInfo graphConfiguration)
-      throws GraphGeneratorException {
-    HadoopFileInputMeta fileInputMeta = new HadoopFileInputMeta();
-    fileInputMeta.setFilenameField("filename");
-    fileInputMeta.setFileName(new String[] { "${csvInputFilePath}" });
-    fileInputMeta.setDefault();
-    fileInputMeta.setEncoding(CarbonCommonConstants.DEFAULT_CHARSET);
-    fileInputMeta.setEnclosure("\"");
-    fileInputMeta.setHeader(true);
-    fileInputMeta.setSeparator(",");
-    fileInputMeta.setAcceptingFilenames(true);
-    fileInputMeta.setAcceptingStepName("getFileNames");
-    fileInputMeta.setFileFormat("mixed");
-    StepMeta csvDataStep = new StepMeta("HadoopFileInputPlugin", (StepMetaInterface) fileInputMeta);
-    csvDataStep.setLocation(100, 100);
-    int copies = Integer.parseInt(instance.getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-        CarbonCommonConstants.DEFAULT_NUMBER_CORES));
-    if (copies > 1) {
-      csvDataStep.setCopies(4);
-    }
-    csvDataStep.setDraw(true);
-    csvDataStep.setDescription("Read raw data from " + GraphGeneratorConstants.CSV_INPUT);
-
-    return csvDataStep;
-  }
-
-  private StepMeta getCSVInputStep(GraphConfigurationInfo graphConfiguration)
-      throws GraphGeneratorException {
-    CsvInputMeta csvInputMeta = new CsvInputMeta();
-    // Init the Filename...
-    csvInputMeta.setFilename("${csvInputFilePath}");
-    csvInputMeta.setDefault();
-    csvInputMeta.setEncoding(CarbonCommonConstants.DEFAULT_CHARSET);
-    csvInputMeta.setEnclosure("\"");
-    csvInputMeta.setHeaderPresent(true);
-    csvInputMeta.setMaxColumns(maxColumns);
-    StepMeta csvDataStep =
-        new StepMeta(GraphGeneratorConstants.CSV_INPUT, (StepMetaInterface) csvInputMeta);
-    csvDataStep.setLocation(100, 100);
-    csvInputMeta.setFilenameField("filename");
-    csvInputMeta.setLazyConversionActive(false);
-    csvInputMeta.setBufferSize(instance.getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
-        CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
-    //set blocks info id
-    csvInputMeta.setBlocksID(this.blocksID);
-    csvInputMeta.setPartitionID(this.partitionID);
-    csvInputMeta.setEscapeCharacter(this.escapeCharacter);
-    csvInputMeta.setQuoteCharacter(this.quoteCharacter);
-    csvInputMeta.setCommentCharacter(this.commentCharacter);
-    csvInputMeta.setRddIteratorKey(this.rddIteratorKey == null ? "" : this.rddIteratorKey);
-    csvDataStep.setDraw(true);
-    csvDataStep.setDescription("Read raw data from " + GraphGeneratorConstants.CSV_INPUT);
-
-    return csvDataStep;
-  }
-
-  private StepMeta getSliceMeregerStep(GraphConfigurationInfo configurationInfo,
-      GraphConfigurationInfo graphjConfigurationForFact) {
-    CarbonSliceMergerStepMeta sliceMerger = new CarbonSliceMergerStepMeta();
-    sliceMerger.setDefault();
-    sliceMerger.setPartitionID(partitionID);
-    sliceMerger.setSegmentId(segmentId);
-    sliceMerger.setTaskNo(taskNo);
-    sliceMerger.setHeirAndKeySize(configurationInfo.getHeirAndKeySizeString());
-    sliceMerger.setMdkeySize(configurationInfo.getMdkeySize());
-    sliceMerger.setMeasureCount(configurationInfo.getMeasureCount());
-    sliceMerger.setTabelName(configurationInfo.getTableName());
-    sliceMerger.setTableName(schemaInfo.getTableName());
-    sliceMerger.setDatabaseName(schemaInfo.getDatabaseName());
-    sliceMerger.setGroupByEnabled(isAutoAggRequest + "");
-    if (isAutoAggRequest) {
-      String[] aggType = configurationInfo.getAggType();
-      StringBuilder builder = new StringBuilder();
-      for (int i = 0; i < aggType.length - 1; i++) {
-        if (aggType[i].equals(CarbonCommonConstants.COUNT)) {
-          builder.append(CarbonCommonConstants.SUM);
-        } else {
-          builder.append(aggType[i]);
-        }
-        builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      }
-      builder.append(aggType[aggType.length - 1]);
-      sliceMerger.setAggregatorString(builder.toString());
-      String[] aggClass = configurationInfo.getAggClass();
-      builder = new StringBuilder();
-      for (int i = 0; i < aggClass.length - 1; i++) {
-        builder.append(aggClass[i]);
-        builder.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      }
-      builder.append(aggClass[aggClass.length - 1]);
-      sliceMerger.setAggregatorClassString(builder.toString());
-    } else {
-      sliceMerger.setAggregatorClassString(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      sliceMerger.setAggregatorString(CarbonCommonConstants.HASH_SPC_CHARACTER);
-    }
-    sliceMerger.setFactDimLensString("");
-    sliceMerger.setLevelAnddataTypeString(configurationInfo.getLevelAnddataType());
-    StepMeta sliceMergerMeta =
-        new StepMeta(GraphGeneratorConstants.CARBON_SLICE_MERGER + configurationInfo.getTableName(),
-            (StepMetaInterface) sliceMerger);
-    sliceMergerMeta.setStepID(GraphGeneratorConstants.CARBON_SLICE_MERGER_ID);
-    xAxixLocation += 120;
-    sliceMergerMeta.setLocation(xAxixLocation, yAxixLocation);
-    sliceMergerMeta.setDraw(true);
-    sliceMergerMeta.setDescription(
-        "SliceMerger: " + GraphGeneratorConstants.CARBON_SLICE_MERGER + configurationInfo
-            .getTableName());
-    return sliceMergerMeta;
-  }
-
-  private DatabaseMeta getDatabaseMeta(GraphConfigurationInfo configurationInfo)
-      throws GraphGeneratorException {
-    return new DatabaseMeta();
-  }
-
-  private StepMeta getTableInputStep(GraphConfigurationInfo configurationInfo)
-      throws GraphGeneratorException {
-    TableInputMeta tableInput = new TableInputMeta();
-    tableInput.setDatabaseMeta(getDatabaseMeta(configurationInfo));
-    tableInput.setSQL(configurationInfo.getTableInputSqlQuery());
-    //
-    StepMeta tableInputStep =
-        new StepMeta(GraphGeneratorConstants.TABLE_INPUT, (StepMetaInterface) tableInput);
-    xAxixLocation += 120;
-    tableInputStep.setLocation(xAxixLocation, yAxixLocation);
-    //
-    tableInputStep.setDraw(true);
-    tableInputStep
-        .setDescription("Read Data From Fact Table: " + GraphGeneratorConstants.TABLE_INPUT);
-
-    return tableInputStep;
-  }
-
-  private StepMeta getCarbonCSVBasedSurrogateKeyStep(GraphConfigurationInfo graphConfiguration) {
-    //
-    CarbonCSVBasedSeqGenMeta seqMeta = new CarbonCSVBasedSeqGenMeta();
-    seqMeta.setPartitionID(partitionID);
-    seqMeta.setSegmentId(segmentId);
-    seqMeta.setTaskNo(taskNo);
-    seqMeta.setCarbondim(graphConfiguration.getDimensionString());
-    seqMeta.setComplexTypeString(graphConfiguration.getComplexTypeString());
-    seqMeta.setColumnPropertiesString(graphConfiguration.getColumnPropertiesString());
-    seqMeta.setBatchSize(Integer.parseInt(graphConfiguration.getBatchSize()));
-    seqMeta.setNoDictionaryDims(graphConfiguration.getNoDictionaryDims());
-    seqMeta.setDimensionColumnsDataType(graphConfiguration.getDimensionColumnsDataType());
-    seqMeta.setTableName(schemaInfo.getTableName());
-    seqMeta.setDatabaseName(schemaInfo.getDatabaseName());
-    seqMeta.setComplexDelimiterLevel1(schemaInfo.getComplexDelimiterLevel1());
-    seqMeta.setComplexDelimiterLevel2(schemaInfo.getComplexDelimiterLevel2());
-    seqMeta.setCarbonmsr(graphConfiguration.getMeasuresString());
-    seqMeta.setCarbonProps(graphConfiguration.getPropertiesString());
-    seqMeta.setCarbonhier(graphConfiguration.getHiersString());
-    seqMeta.setCarbonhierColumn(graphConfiguration.getHierColumnString());
-    seqMeta.setDimensionColumnIds(graphConfiguration.getDimensionColumnIds());
-    seqMeta.setMetaMetaHeirSQLQueries(graphConfiguration.getDimensionSqlQuery());
-    seqMeta.setColumnAndTableNameColumnMapForAggString(
-        graphConfiguration.getColumnAndTableNameColumnMapForAgg());
-    seqMeta.setForgienKeyPrimayKeyString(graphConfiguration.getForgienKeyAndPrimaryKeyMapString());
-    seqMeta.setTableName(graphConfiguration.getTableName());
-    seqMeta.setDateFormat(dateFormat);
-    seqMeta.setModifiedDimension(modifiedDimension);
-    seqMeta.setForeignKeyHierarchyString(graphConfiguration.getForeignKeyHierarchyString());
-    seqMeta.setPrimaryKeysString(graphConfiguration.getPrimaryKeyString());
-    seqMeta.setCarbonMeasureNames(graphConfiguration.getMeasureNamesString());
-    seqMeta.setHeirNadDimsLensString(graphConfiguration.getHeirAndDimLens());
-    seqMeta.setActualDimNames(graphConfiguration.getActualDimensionColumns());
-    seqMeta.setNormHiers(graphConfiguration.getNormHiers());
-    seqMeta.setHeirKeySize(graphConfiguration.getHeirAndKeySizeString());
-    seqMeta.setColumnSchemaDetails(graphConfiguration.getColumnSchemaDetails().toString());
-    seqMeta.setTableOption(graphConfiguration.getTableOptionWrapper().toString());
-    String[] aggType = graphConfiguration.getAggType();
-    StringBuilder builder = new StringBuilder();
-    for (int i = 0; i < aggType.length; i++) {
-      builder.append(aggType[i]);
-      builder.append(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER);
-    }
-    seqMeta.setMsrAggregatorString(builder.toString());
-
-    seqMeta.setDriverClass(graphConfiguration.getDriverclass());
-    seqMeta.setConnectionURL(graphConfiguration.getConnectionUrl());
-    seqMeta.setUserName(graphConfiguration.getUsername());
-    seqMeta.setPassword(graphConfiguration.getPassword());
-    seqMeta.setMeasureDataType(graphConfiguration.getMeasureDataTypeInfo());
-    seqMeta.setDenormColumNames(graphConfiguration.getDenormColumns());
-    seqMeta.setAggregate(graphConfiguration.isAGG());
-    seqMeta.setTableNames(graphConfiguration.getDimensionTableNames());
-    StepMeta mdkeyStepMeta = new StepMeta(GraphGeneratorConstants.CARBON_SURROGATE_KEY_GENERATOR,
-        (StepMetaInterface) seqMeta);
-    mdkeyStepMeta.setStepID(GraphGeneratorConstants.CARBON_CSV_BASED_SURROAGATEGEN_ID);
-    xAxixLocation += 120;
-    //
-    mdkeyStepMeta.setLocation(xAxixLocation, yAxixLocation);
-    mdkeyStepMeta.setDraw(true);
-    mdkeyStepMeta.setDescription("Generate Surrogate For Table Data: "
-        + GraphGeneratorConstants.CARBON_SURROGATE_KEY_GENERATOR);
-    return mdkeyStepMeta;
-  }
-
-  private StepMeta getMDKeyStep(GraphConfigurationInfo graphConfiguration) {
-    MDKeyGenStepMeta carbonMdKey = new MDKeyGenStepMeta();
-    carbonMdKey.setIsUseInvertedIndex(
-        NonDictionaryUtil.convertBooleanArrToString(graphConfiguration.getIsUseInvertedIndex()));
-    carbonMdKey.setPartitionID(partitionID);
-    carbonMdKey.setSegmentId(segmentId);
-    carbonMdKey.setNumberOfCores(graphConfiguration.getNumberOfCores());
-    carbonMdKey.setTableName(graphConfiguration.getTableName());
-    carbonMdKey.setDatabaseName(schemaInfo.getDatabaseName());
-    carbonMdKey.setTableName(schemaInfo.getTableName());
-    carbonMdKey.setComplexTypeString(graphConfiguration.getComplexTypeString());
-    carbonMdKey.setAggregateLevels(CarbonDataProcessorUtil
-        .getLevelCardinalitiesString(graphConfiguration.getDimCardinalities(),
-            graphConfiguration.getDimensions()));
-    carbonMdKey.setNoDictionaryDimsMapping(NonDictionaryUtil
-        .convertBooleanArrToString(graphConfiguration.getIsNoDictionaryDimMapping()));
-    carbonMdKey.setMeasureCount(graphConfiguration.getMeasureCount() + "");
-    carbonMdKey.setColumnGroupsString(graphConfiguration.getColumnGroupsString());
-    carbonMdKey.setDimensionCount(graphConfiguration.getActualDims().length + "");
-    carbonMdKey.setComplexDimsCount(graphConfiguration.getComplexTypeString().isEmpty() ?
-        "0" :
-        graphConfiguration.getComplexTypeString()
-            .split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER).length + "");
-    carbonMdKey.setMeasureDataType(graphConfiguration.getMeasureDataTypeInfo());
-    carbonMdKey.setTaskNo(taskNo);
-    carbonMdKey.setFactTimeStamp(factTimeStamp);
-    StepMeta mdkeyStepMeta =
-        new StepMeta(GraphGeneratorConstants.MDKEY_GENERATOR + graphConfiguration.getTableName(),
-            (StepMetaInterface) carbonMdKey);
-    mdkeyStepMeta
-        .setName(GraphGeneratorConstants.MDKEY_GENERATOR_ID + graphConfiguration.getTableName());
-    mdkeyStepMeta.setStepID(GraphGeneratorConstants.MDKEY_GENERATOR_ID);
-    //
-    xAxixLocation += 120;
-    mdkeyStepMeta.setLocation(xAxixLocation, yAxixLocation);
-    mdkeyStepMeta.setDraw(true);
-    mdkeyStepMeta.setDescription(
-        "Generate MDKey For Table Data: " + GraphGeneratorConstants.MDKEY_GENERATOR
-            + graphConfiguration.getTableName());
-    carbonMdKey.setNoDictionaryDims(graphConfiguration.getNoDictionaryDims());
-
-    return mdkeyStepMeta;
-  }
-
-  private StepMeta getSelectValueToChangeTheDataType(GraphConfigurationInfo graphConfiguration,
-      int counter) {
-    //
-    SelectValuesMeta selectValues = new SelectValuesMeta();
-    selectValues.allocate(0, 0, 0);
-    StepMeta selectValueMeta = new StepMeta(
-        GraphGeneratorConstants.SELECT_REQUIRED_VALUE + "Change Dimension And Measure DataType"
-            + System.currentTimeMillis() + counter, (StepMetaInterface) selectValues);
-    xAxixLocation += 120;
-    selectValueMeta.setName("SelectValueToChangeChangeData");
-    selectValueMeta.setLocation(xAxixLocation, yAxixLocation);
-    selectValueMeta.setDraw(true);
-    selectValueMeta.setDescription(
-        "Change The Data Type For Measures: " + GraphGeneratorConstants.SELECT_REQUIRED_VALUE);
-
-    String inputQuery = graphConfiguration.getTableInputSqlQuery();
-    String[] columns = parseQueryAndReturnColumns(inputQuery);
-
-    SelectMetadataChange[] changeMeta = new SelectMetadataChange[columns.length];
-    Map<String, Boolean> measureDatatypeMap =
-        getMeasureDatatypeMap(graphConfiguration.getMeasureDataTypeInfo());
-    String[] measures = graphConfiguration.getMeasures();
-    String dimensionString = graphConfiguration.getActualDimensionColumns();
-    String[] dimension = dimensionString.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-    for (int i = 0; i < columns.length; i++) {
-      changeMeta[i] = new SelectMetadataChange(selectValues);
-      changeMeta[i].setName(columns[i]);
-      changeMeta[i].setType(2);
-      if (isMeasureColumn(measures, columns[i]) && isNotDimesnionColumn(dimension, columns[i])) {
-        Boolean isString = measureDatatypeMap.get(columns[i]);
-        if (isString != null && isString) {
-          changeMeta[i].setType(2);
-        } else {
-          changeMeta[i].setType(6);
-        }
-      }
-      changeMeta[i].setStorageType(0);
-    }
-    //
-    selectValues.setMeta(changeMeta);
-    return selectValueMeta;
-  }
-
-  private boolean isMeasureColumn(String[] measures, String column) {
-    for (int i = 0; i < measures.length; i++) {
-      if (measures[i].equals(column)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  private boolean isNotDimesnionColumn(String[] dimension, String column) {
-    for (int i = 0; i < dimension.length; i++) {
-      if (dimension[i].equals(column)) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-  private Map<String, Boolean> getMeasureDatatypeMap(String measureDataType) {
-    if (measureDataType == null || "".equals(measureDataType)) {
-      return new HashMap<String, Boolean>(1);
-    }
-    Map<String, Boolean> resultMap = new HashMap<String, Boolean>(1);
-
-    String[] measures = measureDataType.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    String[] measureValue = null;
-    for (int i = 0; i < measures.length; i++) {
-      measureValue = measures[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      resultMap.put(measureValue[0], Boolean.valueOf(measureValue[1]));
-    }
-    return resultMap;
-  }
-
-  /**
-   * @param inputQuery
-   * @return
-   */
-  private String[] parseQueryAndReturnColumns(String inputQuery) {
-    Set<String> cols = new LinkedHashSet<String>();
-    String columnString =
-        inputQuery.substring(inputQuery.indexOf("SELECT") + 6, inputQuery.indexOf("FROM"));
-    String[] columns = columnString.split(",");
-    for (int i = 0; i < columns.length; i++) {
-      if (columns[i].indexOf("\"") > -1) {
-        columns[i] = columns[i].replace("\"", "");
-        if (columns[i].contains(".")) {
-          columns[i] = columns[i].split("\\.")[1];
-        }
-      }
-
-      cols.add(columns[i].replaceAll(System.getProperty("line.separator"), "").trim());
-    }
-    return cols.toArray(new String[cols.size()]);
-  }
-
-  private StepMeta getSortStep(GraphConfigurationInfo graphConfiguration)
-      throws GraphGeneratorException {
-    String[] actualMeasures = graphConfiguration.getMeasures();
-
-    SortKeyStepMeta sortRowsMeta = new SortKeyStepMeta();
-    sortRowsMeta.setPartitionID(partitionID);
-    sortRowsMeta.setSegmentId(segmentId);
-    sortRowsMeta.setTaskNo(taskNo);
-    sortRowsMeta.setTabelName(graphConfiguration.getTableName());
-    sortRowsMeta.setTableName(schemaInfo.getTableName());
-    sortRowsMeta.setDatabaseName(schemaInfo.getDatabaseName());
-    sortRowsMeta.setOutputRowSize(actualMeasures.length + 1 + "");
-    sortRowsMeta.setDimensionCount(graphConfiguration.getDimensions().length + "");
-    sortRowsMeta.setComplexDimensionCount(graphConfiguration.getComplexTypeString().isEmpty() ?
-        "0" :
-        graphConfiguration.getComplexTypeString()
-            .split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER).length + "");
-    sortRowsMeta.setIsUpdateMemberRequest(isUpdateMemberRequest + "");
-    sortRowsMeta.setMeasureCount(graphConfiguration.getMeasureCount() + "");
-    sortRowsMeta.setNoDictionaryDims(graphConfiguration.getNoDictionaryDims());
-    sortRowsMeta.setMeasureDataType(graphConfiguration.getMeasureDataTypeInfo());
-    sortRowsMeta.setNoDictionaryDimsMapping(NonDictionaryUtil
-        .convertBooleanArrToString(graphConfiguration.getIsNoDictionaryDimMapping()));
-
-    StepMeta sortRowsStep = new StepMeta(
-        GraphGeneratorConstants.SORT_KEY_AND_GROUPBY + graphConfiguration.getTableName(),
-        (StepMetaInterface) sortRowsMeta);
-
-    xAxixLocation += 120;
-    sortRowsStep.setDraw(true);
-    sortRowsStep.setLocation(xAxixLocation, yAxixLocation);
-    sortRowsStep.setStepID(GraphGeneratorConstants.SORTKEY_ID);
-    sortRowsStep.setDescription(
-        "Sort Key: " + GraphGeneratorConstants.SORT_KEY + graphConfiguration.getTableName());
-    sortRowsStep.setName(
-        "Sort Key: " + GraphGeneratorConstants.SORT_KEY + graphConfiguration.getTableName());
-    return sortRowsStep;
-  }
-
-  private GraphConfigurationInfo getGraphConfigInfoForFact(
-      CarbonDataLoadSchema carbonDataLoadSchema) throws GraphGeneratorException {
-    //
-    GraphConfigurationInfo graphConfiguration = new GraphConfigurationInfo();
-    List<CarbonDimension> dimensions = carbonDataLoadSchema.getCarbonTable()
-        .getDimensionByTableName(carbonDataLoadSchema.getCarbonTable().getFactTableName());
-    prepareIsUseInvertedIndex(dimensions, graphConfiguration);
-    graphConfiguration
-        .setDimensions(CarbonSchemaParser.getTableDimensions(dimensions, carbonDataLoadSchema));
-    graphConfiguration
-        .setActualDims(CarbonSchemaParser.getTableDimensions(dimensions, carbonDataLoadSchema));
-    graphConfiguration
-        .setColumnPropertiesString(CarbonSchemaParser.getColumnPropertiesString(dimensions));
-    graphConfiguration.setComplexTypeString(CarbonSchemaParser.getComplexTypeString(dimensions));
-    prepareNoDictionaryMapping(dimensions, graphConfiguration);
-    graphConfiguration
-        .setColumnSchemaDetails(CarbonSchemaParser.getColumnSchemaDetails(dimensions));
-    graphConfiguration.setTableOptionWrapper(getTableOptionWrapper());
-    String factTableName = carbonDataLoadSchema.getCarbonTable().getFactTableName();
-    graphConfiguration.setTableName(factTableName);
-    StringBuilder dimString = new StringBuilder();
-    //
-    int currentCount =
-        CarbonSchemaParser.getDimensionString(dimensions, dimString, 0, carbonDataLoadSchema);
-    StringBuilder noDictionarydimString = new StringBuilder();
-    CarbonSchemaParser
-        .getNoDictionaryDimensionString(dimensions, noDictionarydimString, 0, carbonDataLoadSchema);
-    graphConfiguration.setNoDictionaryDims(noDictionarydimString.toString());
-
-    String tableString =
-        CarbonSchemaParser.getTableNameString(dimensions, carbonDataLoadSchema);
-    String dimensionColumnIds = CarbonSchemaParser.getColumnIdString(dimensions);
-    graphConfiguration.setDimensionTableNames(tableString);
-    graphConfiguration.setDimensionString(dimString.toString());
-    graphConfiguration.setDimensionColumnIds(dimensionColumnIds);
-    graphConfiguration
-        .setForignKey(CarbonSchemaParser.getForeignKeyForTables(dimensions, carbonDataLoadSchema));
-    List<CarbonMeasure> measures = carbonDataLoadSchema.getCarbonTable()
-        .getMeasureByTableName(carbonDataLoadSchema.getCarbonTable().getFactTableName());
-    graphConfiguration
-        .setMeasuresString(CarbonSchemaParser.getMeasureString(measures, currentCount));
-    graphConfiguration
-        .setHiersString(CarbonSchemaParser.getHierarchyString(dimensions, carbonDataLoadSchema));
-    graphConfiguration.setHierColumnString(
-        CarbonSchemaParser.getHierarchyStringWithColumnNames(dimensions, carbonDataLoadSchema));
-    graphConfiguration.setMeasureUniqueColumnNamesString(
-        CarbonSchemaParser.getMeasuresUniqueColumnNamesString(measures));
-    graphConfiguration.setForeignKeyHierarchyString(CarbonSchemaParser
-        .getForeignKeyHierarchyString(dimensions, carbonDataLoadSchema, factTableName));
-    graphConfiguration.setConnectionName("target");
-    graphConfiguration.setHeirAndDimLens(
-        CarbonSchemaParser.getHeirAndCardinalityString(dimensions, carbonDataLoadSchema));
-    //setting dimension store types
-    graphConfiguration.setColumnGroupsString(CarbonSchemaParser.getColumnGroups(dimensions));
-    graphConfiguration.setPrimaryKeyString(
-        CarbonSchemaParser.getPrimaryKeyString(dimensions, carbonDataLoadSchema));
-    graphConfiguration
-        .setDenormColumns(CarbonSchemaParser.getDenormColNames(dimensions, carbonDataLoadSchema));
-
-    graphConfiguration.setLevelAnddataType(
-        CarbonSchemaParser.getLevelAndDataTypeMapString(dimensions, carbonDataLoadSchema));
-
-    graphConfiguration.setForgienKeyAndPrimaryKeyMapString(CarbonSchemaParser
-        .getForeignKeyAndPrimaryKeyMapString(carbonDataLoadSchema.getDimensionRelationList()));
-
-    graphConfiguration.setMdkeySize(CarbonSchemaParser.getMdkeySizeForFact(dimensions));
-    Set<String> measureColumn = new HashSet<String>(measures.size());
-    for (int i = 0; i < measures.size(); i++) {
-      measureColumn.add(measures.get(i).getColName());
-    }
-    char[] type = new char[measureColumn.size()];
-    Arrays.fill(type, 'n');
-    graphConfiguration.setType(type);
-    graphConfiguration.setMeasureCount(measureColumn.size() + "");
-    graphConfiguration.setHeirAndKeySizeString(
-        CarbonSchemaParser.getHeirAndKeySizeMapForFact(dimensions, carbonDataLoadSchema));
-    graphConfiguration.setAggType(CarbonSchemaParser.getMeasuresAggragatorArray(measures));
-    graphConfiguration.setMeasureNamesString(CarbonSchemaParser.getMeasuresNamesString(measures));
-    graphConfiguration
-        .setActualDimensionColumns(CarbonSchemaParser.getActualDimensions(dimensions));
-    graphConfiguration
-        .setDimensionColumnsDataType(CarbonSchemaParser.getDimensionsDataTypes(dimensions));
-    //graphConfiguration.setNormHiers(CarbonSchemaParser.getNormHiers(table, schema));
-    graphConfiguration.setMeasureDataTypeInfo(CarbonSchemaParser.getMeasuresDataType(measures));
-    graphConfiguration.setStoreLocation(
-        this.databaseName + '/' + carbonDataLoadSchema.getCarbonTable().getFactTableName());
-    graphConfiguration.setBlockletSize(
-        (instance.getProperty("com.huawei.unibi.carbon.blocklet.size", DEFAUL_BLOCKLET_SIZE)));
-    graphConfiguration.setMaxBlockletInFile(
-        (instance.getProperty("carbon.max.blocklet.in.file", DEFAULE_MAX_BLOCKLET_IN_FILE)));
-    graphConfiguration.setNumberOfCores(
-        (instance.getProperty(CarbonCommonConstants.NUM_CORES_LOADING, DEFAULT_NUMBER_CORES)));
-
-    // check quotes required in query or Not
-    boolean isQuotesRequired = true;
-    String quote = CarbonSchemaParser.QUOTES;
-    graphConfiguration.setTableInputSqlQuery(CarbonSchemaParser
-        .getTableInputSQLQuery(dimensions, measures,
-            carbonDataLoadSchema.getCarbonTable().getFactTableName(), isQuotesRequired,
-            carbonDataLoadSchema));
-    graphConfiguration
-        .setBatchSize((instance.getProperty("carbon.batch.size", DEFAULT_BATCH_SIZE)));
-    graphConfiguration.setSortSize((instance.getProperty("carbon.sort.size", DEFAULT_SORT_SIZE)));
-    graphConfiguration.setDimensionSqlQuery(CarbonSchemaParser
-        .getDimensionSQLQueries(dimensions, carbonDataLoadSchema, isQuotesRequired, quote));
-    graphConfiguration.setMetaHeirString(
-        CarbonSchemaParser.getMetaHeirString(dimensions, carbonDataLoadSchema.getCarbonTable()));
-    graphConfiguration
-        .setDimCardinalities(CarbonSchemaParser.getCardinalities(dimensions, carbonDataLoadSchema));
-
-    graphConfiguration.setMeasures(CarbonSchemaParser.getMeasures(measures));
-    graphConfiguration.setAGG(false);
-    return graphConfiguration;
-  }
-
-  /**
-   * the method returns the table option wrapper
-   *
-   * @return
-   */
-  private TableOptionWrapper getTableOptionWrapper() {
-    TableOptionWrapper tableOptionWrapper = TableOptionWrapper.getTableOptionWrapperInstance();
-    tableOptionWrapper.setTableOption(schemaInfo.getSerializationNullFormat());
-    tableOptionWrapper.setTableOption(schemaInfo.getBadRecordsLoggerEnable());
-    tableOptionWrapper.setTableOption(schemaInfo.getBadRecordsLoggerAction());
-    return tableOptionWrapper;
-  }
-
-  public CarbonTable getTable() {
-    return carbonDataLoadSchema.getCarbonTable();
-  }
-
-  /**
-   * Preparing the boolean [] to map whether the dimension is no Dictionary or not.
-   *
-   * @param dims
-   * @param graphConfig
-   */
-  private void prepareNoDictionaryMapping(List<CarbonDimension> dims,
-      GraphConfigurationInfo graphConfig) {
-    List<Boolean> noDictionaryMapping = new ArrayList<Boolean>();
-    for (CarbonDimension dimension : dims) {
-      // for  complex type need to break the loop
-      if (dimension.getNumberOfChild() > 0) {
-        break;
-      }
-
-      if (!dimension.getEncoder().contains(Encoding.DICTIONARY)) {
-        noDictionaryMapping.add(true);
-        //NoDictionaryMapping[index] = true;
-      } else {
-        noDictionaryMapping.add(false);
-      }
-    }
-
-    graphConfig.setIsNoDictionaryDimMapping(
-        noDictionaryMapping.toArray(new Boolean[noDictionaryMapping.size()]));
-  }
-
-  /**
-   * Preparing the boolean [] to map whether the dimension use inverted index or not.
-   *
-   * @param dims
-   * @param graphConfig
-   */
-  private void prepareIsUseInvertedIndex(List<CarbonDimension> dims,
-      GraphConfigurationInfo graphConfig) {
-    List<Boolean> isUseInvertedIndexList = new ArrayList<Boolean>();
-    for (CarbonDimension dimension : dims) {
-      if (dimension.isUseInvertedIndex()) {
-        isUseInvertedIndexList.add(true);
-      } else {
-        isUseInvertedIndexList.add(false);
-      }
-    }
-    graphConfig.setIsUseInvertedIndex(
-        isUseInvertedIndexList.toArray(new Boolean[isUseInvertedIndexList.size()]));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorConstants.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorConstants.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorConstants.java
deleted file mode 100644
index 95a7823..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorConstants.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.graphgenerator;
-
-public final class GraphGeneratorConstants {
-  /**
-   * TABLE_INPUT
-   */
-  public static final String TABLE_INPUT = "Table Input";
-  /**
-   * CARBON_SURROGATE_KEY_GENERATOR
-   */
-  public static final String CARBON_SURROGATE_KEY_GENERATOR = "Carbon Surrogate Key Generator";
-  /**
-   * MDKEY_GENERATOR
-   */
-  public static final String MDKEY_GENERATOR = "MDKey Generator";
-  /**
-   * SORT_KEY
-   */
-  public static final String SORT_KEY = "Sort keys";
-  /**
-   * SORT_KEY
-   */
-  public static final String SORT_KEY_AND_GROUPBY = "Sort keys And Group By Step";
-  /**
-   * CARBON_SLICE_MERGER
-   */
-  public static final String CARBON_SLICE_MERGER = "Carbon Slice Merger";
-  /**
-   * SELECT_REQUIRED_VALUE
-   */
-  public static final String SELECT_REQUIRED_VALUE = "Select Required Value";
-  /**
-   * CSV Input
-   */
-  public static final String CSV_INPUT = "CSV Input";
-  /**
-   * CARBON_MDKEY_GENERATOR_ID
-   */
-  public static final String MDKEY_GENERATOR_ID = "MDKeyGen";
-  /**
-   * CARBON_SLICE_MERGER_ID
-   */
-  public static final String CARBON_SLICE_MERGER_ID = "CarbonSliceMerger";
-  /**
-   * CARBON_SLICE_MERGER_ID
-   */
-  public static final String SORTKEY_ID = "SortKey";
-  /**
-   * CARBON_CSV_BASED_SURROAGATEGEN_ID
-   */
-  public static final String CARBON_CSV_BASED_SURROAGATEGEN_ID = "CarbonCSVBasedSurrogateGen";
-
-  private GraphGeneratorConstants() {
-
-  }
-
-}



[07/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorException.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorException.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorException.java
deleted file mode 100644
index 6a35994..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGeneratorException.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.graphgenerator;
-
-import java.util.Locale;
-
-public class GraphGeneratorException 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 GraphGeneratorException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param msg The error message for this exception.
-   */
-  public GraphGeneratorException(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/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index f389a44..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
+++ /dev/null
@@ -1,894 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.graphgenerator.configuration;
-
-import java.util.Map;
-
-import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper;
-import org.apache.carbondata.processing.schema.metadata.TableOptionWrapper;
-
-public class GraphConfigurationInfo {
-  private String connectionName;
-
-  private String dbType;
-
-  private String numberOfCores;
-
-  private String storeLocation;
-
-  private String tableName;
-
-  private String blockletSize;
-
-  private String maxBlockletInFile;
-
-  private String batchSize;
-
-  private Map<String, String> dimCardinalities;
-
-  private String[] dimensions;
-
-  private String noDictionaryDims;
-
-  private String[] measures;
-
-  private String dimensionString;
-
-  private String hiersString;
-
-  private String measuresString;
-
-  private String propertiesString;
-
-  private String timeHeirString;
-
-  private String metaHeirString;
-
-  private String metaHeirQueryString;
-
-  private String jndiName;
-
-  private Map<String, String> tableMeasuresAndDataTypeMap;
-
-  private String tableInputSqlQuery;
-
-  private String dimensionSqlQuery;
-
-  private String sortSize;
-
-  private boolean isAGG;
-
-  private String driverclass;
-
-  private String username;
-
-  private String password;
-
-  private String connectionUrl;
-
-  private String[] actualDims;
-
-  /**
-   * Sets the dimension:hirearchy#levelnames1,levelName2
-   */
-  private String dimensionTableNames;
-
-  /**
-   * column Ids concatenated by a delimeter
-   */
-  private String dimensionColumnIds;
-
-  /**
-   * Agg type
-   */
-  private String[] aggType;
-
-  /**
-   * mdkeySize
-   */
-  private String mdkeySize;
-
-  /**
-   * complexTypeString
-   */
-  private String complexTypeString;
-
-  /**
-   * measureCount
-   */
-
-  private String measureCount;
-
-  /**
-   * heirAndKeySizeString
-   */
-  private String heirAndKeySizeString;
-
-  /**
-   * hier and containing columns string
-   */
-  private String hierColumnString;
-
-  /**
-   * forignKey
-   */
-  private String[] forignKey;
-
-  /**
-   * Foreign Key and respective hierarchy
-   */
-  private String foreignKeyHierarchyString;
-
-  /**
-   * Primary key String
-   */
-  private String primaryKeyString;
-
-  /**
-   * Measure Names
-   */
-  private String measureNamesString;
-
-  /**
-   * Measure Names
-   */
-  private String measureUniqueColumnNamesString;
-
-  /**
-   * actualDimensionColumns
-   */
-  private String actualDimensionColumns;
-
-  /**
-   * normHiers
-   */
-  private String normHiers;
-
-  private String forgienKeyAndPrimaryKeyMapString;
-
-  /**
-   * heirAndDimLens
-   */
-  private String heirAndDimLens;
-
-  /**
-   * measureTypeInfo
-   */
-  private String measureDataTypeInfo;
-
-  /**
-   * columnAndTableName_ColumnMapForAgg
-   */
-  private String columnAndTableNameColumnMapForAgg;
-  /**
-   * denormColumns
-   */
-  private String denormColumns;
-  private String[] aggClass;
-  /**
-   * type
-   */
-  private char[] type;
-  private String levelAnddataType;
-
-  private Boolean[] isNoDictionaryDimMapping;
-
-  private Boolean[] isUseInvertedIndex;
-
-  private String columnPropertiesString;
-
-  /**
-   * wrapper object holding the columnschemadetails
-   */
-  private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper;
-
-  /**
-   * wrapper object holding the table options details needed while dataload
-   */
-  private TableOptionWrapper tableOptionWrapper;
-
-  /**
-   * It is column groups in below format
-   * 0,1~2~3,4,5,6~7~8,9
-   * groups are
-   * ,-> all ordinal with different group id
-   * ~-> all ordinal with same group id
-   */
-  private String columnGroupsString;
-  private String columnsDataTypeString;
-  /**
-   * @return isUseInvertedIndex
-   */
-  public Boolean[] getIsUseInvertedIndex() {
-    return isUseInvertedIndex;
-  }
-
-  /**
-   * @param isUseInvertedIndex the bool array whether use inverted index to set
-   */
-  public void setIsUseInvertedIndex(Boolean[] isUseInvertedIndex) {
-    this.isUseInvertedIndex = isUseInvertedIndex;
-  }
-
-  /**
-   * @param connectionName the connectionName to set
-   */
-  public void setConnectionName(String connectionName) {
-    this.connectionName = connectionName;
-  }
-
-  /**
-   * @return the numberOfCores
-   */
-  public String getNumberOfCores() {
-    return numberOfCores;
-  }
-
-  /**
-   * @param numberOfCores the numberOfCores to set
-   */
-  public void setNumberOfCores(String numberOfCores) {
-    this.numberOfCores = numberOfCores;
-  }
-
-  /**
-   * @param storeLocation the storeLocation to set
-   */
-  public void setStoreLocation(String storeLocation) {
-    this.storeLocation = storeLocation;
-  }
-
-  /**
-   * @return the tableName
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * @param tableName the tableName to set
-   */
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  public String getComplexTypeString() {
-    return complexTypeString;
-  }
-
-  public void setComplexTypeString(String complexTypeString) {
-    this.complexTypeString = complexTypeString;
-  }
-
-  /**
-   * @param blockletSize the blockletSize to set
-   */
-  public void setBlockletSize(String blockletSize) {
-    this.blockletSize = blockletSize;
-  }
-
-  /**
-   * @param maxBlockletInFile the maxBlockletInFile to set
-   */
-  public void setMaxBlockletInFile(String maxBlockletInFile) {
-    this.maxBlockletInFile = maxBlockletInFile;
-  }
-
-  /**
-   * @return the batchSize
-   */
-  public String getBatchSize() {
-    return batchSize;
-  }
-
-  /**
-   * @param batchSize the batchSize to set
-   */
-  public void setBatchSize(String batchSize) {
-    this.batchSize = batchSize;
-  }
-
-  /**
-   * @return the dimCardinalities
-   */
-  public Map<String, String> getDimCardinalities() {
-    return dimCardinalities;
-  }
-
-  /**
-   * @param dimCardinalities the dimCardinalities to set
-   */
-  public void setDimCardinalities(Map<String, String> dimCardinalities) {
-    this.dimCardinalities = dimCardinalities;
-  }
-
-  /**
-   * @return the dimensions
-   */
-  public String[] getDimensions() {
-    return dimensions;
-  }
-
-  /**
-   * @param dimensions the dimensions to set
-   */
-  public void setDimensions(String[] dimensions) {
-    this.dimensions = dimensions;
-  }
-
-  /**
-   * @return the measures
-   */
-  public String[] getMeasures() {
-    return measures;
-  }
-
-  /**
-   * @param measures the measures to set
-   */
-  public void setMeasures(String[] measures) {
-    this.measures = measures;
-  }
-
-  /**
-   * @return the dimensionString
-   */
-  public String getDimensionString() {
-    return dimensionString;
-  }
-
-  /**
-   * @param dimensionString the dimensionString to set
-   */
-  public void setDimensionString(String dimensionString) {
-    this.dimensionString = dimensionString;
-  }
-
-  /**
-   * getNormHiers
-   *
-   * @return String
-   */
-  public String getNormHiers() {
-    return normHiers;
-  }
-
-  /**
-   * @return the hiersString
-   */
-  public String getHiersString() {
-    return hiersString;
-  }
-
-  /**
-   * @param hiersString the hiersString to set
-   */
-  public void setHiersString(String hiersString) {
-    this.hiersString = hiersString;
-  }
-
-  /**
-   * @return the measuresString
-   */
-  public String getMeasuresString() {
-    return measuresString;
-  }
-
-  /**
-   * @param measuresString the measuresString to set
-   */
-  public void setMeasuresString(String measuresString) {
-    this.measuresString = measuresString;
-  }
-
-  /**
-   * @return the propertiesString
-   */
-  public String getPropertiesString() {
-    return propertiesString;
-  }
-
-  /**
-   * @return the metaHeirString
-   */
-  public String getMetaHeirString() {
-    return metaHeirString;
-  }
-
-  /**
-   * @param metaHeirString the metaHeirString to set
-   */
-  public void setMetaHeirString(String metaHeirString) {
-    this.metaHeirString = metaHeirString;
-  }
-
-  /**
-   * @return the tableInputSqlQuery
-   */
-  public String getTableInputSqlQuery() {
-    return tableInputSqlQuery;
-  }
-
-  /**
-   * @param tableInputSqlQuery the tableInputSqlQuery to set
-   */
-  public void setTableInputSqlQuery(String tableInputSqlQuery) {
-    this.tableInputSqlQuery = tableInputSqlQuery;
-  }
-
-  /**
-   * @return the dimensionSqlQuery
-   */
-  public String getDimensionSqlQuery() {
-    return dimensionSqlQuery;
-  }
-
-  /**
-   * @param dimensionSqlQuery the dimensionSqlQuery to set
-   */
-  public void setDimensionSqlQuery(String dimensionSqlQuery) {
-    this.dimensionSqlQuery = dimensionSqlQuery;
-  }
-
-  /**
-   * @param sortSize the sortSize to set
-   */
-  public void setSortSize(String sortSize) {
-    this.sortSize = sortSize;
-  }
-
-  /**
-   * @return the isAGG
-   */
-  public boolean isAGG() {
-    return isAGG;
-  }
-
-  /**
-   * @param isAGG the isAGG to set
-   */
-  public void setAGG(boolean isAGG) {
-    this.isAGG = isAGG;
-  }
-
-  /**
-   * @return the driverclass
-   */
-  public String getDriverclass() {
-    return driverclass;
-  }
-
-  /**
-   * @param driverclass the driverclass to set
-   */
-  public void setDriverclass(String driverclass) {
-    this.driverclass = driverclass;
-  }
-
-  /**
-   * @return the username
-   */
-  public String getUsername() {
-    return username;
-  }
-
-  /**
-   * @param username the username to set
-   */
-  public void setUsername(String username) {
-    this.username = username;
-  }
-
-  /**
-   * @return the password
-   */
-  public String getPassword() {
-    return password;
-  }
-
-  /**
-   * @param password the password to set
-   */
-  public void setPassword(String password) {
-    this.password = password;
-  }
-
-  /**
-   * @return the connectionUrl
-   */
-  public String getConnectionUrl() {
-    return connectionUrl;
-  }
-
-  /**
-   * @param connectionUrl the connectionUrl to set
-   */
-  public void setConnectionUrl(String connectionUrl) {
-    this.connectionUrl = connectionUrl;
-  }
-
-  /**
-   * @return the actualDims
-   */
-  public String[] getActualDims() {
-    return actualDims;
-  }
-
-  /**
-   * @param actualDims the actualDims to set
-   */
-  public void setActualDims(String[] actualDims) {
-    this.actualDims = actualDims;
-  }
-
-  /**
-   * @return the dimensionTableNames
-   */
-  public String getDimensionTableNames() {
-    return dimensionTableNames;
-  }
-
-  /**
-   * @param dimensionTableNames the dimensionTableNames to set
-   */
-  public void setDimensionTableNames(String dimensionTableNames) {
-    this.dimensionTableNames = dimensionTableNames;
-  }
-
-  /**
-   * @return
-   */
-  public String getDimensionColumnIds() {
-    return dimensionColumnIds;
-  }
-
-  /**
-   * @param dimensionColumnIds column Ids for dimensions in a table
-   */
-  public void setDimensionColumnIds(String dimensionColumnIds) {
-    this.dimensionColumnIds = dimensionColumnIds;
-  }
-
-  /**
-   * getMdkeySize
-   *
-   * @return String
-   */
-  public String getMdkeySize() {
-    return mdkeySize;
-  }
-
-  /**
-   * setMdkeySize
-   *
-   * @param mdkeySize void
-   */
-  public void setMdkeySize(String mdkeySize) {
-    this.mdkeySize = mdkeySize;
-  }
-
-  /**
-   * getMeasureCount
-   *
-   * @return String
-   */
-  public String getMeasureCount() {
-    return measureCount;
-  }
-
-  /**
-   * setMeasureCount
-   *
-   * @param measureCount void
-   */
-  public void setMeasureCount(String measureCount) {
-    this.measureCount = measureCount;
-  }
-
-  /**
-   * getHeirAndKeySizeString
-   *
-   * @return String
-   */
-  public String getHeirAndKeySizeString() {
-    return heirAndKeySizeString;
-  }
-
-  /**
-   * setHeirAndKeySizeString
-   *
-   * @param heirAndKeySizeString void
-   */
-  public void setHeirAndKeySizeString(String heirAndKeySizeString) {
-    this.heirAndKeySizeString = heirAndKeySizeString;
-  }
-
-  /**
-   * @return Returns the hierColumnString.
-   */
-  public String getHierColumnString() {
-    return hierColumnString;
-  }
-
-  /**
-   * @param hierColumnString The hierColumnString to set.
-   */
-  public void setHierColumnString(String hierColumnString) {
-    this.hierColumnString = hierColumnString;
-  }
-
-  /**
-   * @param forignKey The forignKey to set.
-   */
-  public void setForignKey(String[] forignKey) {
-    this.forignKey = forignKey;
-  }
-
-  /**
-   * @return Returns the foreignKeyHierarchyString.
-   */
-  public String getForeignKeyHierarchyString() {
-    return foreignKeyHierarchyString;
-  }
-
-  /**
-   * @param foreignKeyHierarchyString The foreignKeyHierarchyString to set.
-   */
-  public void setForeignKeyHierarchyString(String foreignKeyHierarchyString) {
-    this.foreignKeyHierarchyString = foreignKeyHierarchyString;
-  }
-
-  /**
-   * @return Returns the primaryKeyString.
-   */
-  public String getPrimaryKeyString() {
-    return primaryKeyString;
-  }
-
-  /**
-   * @param primaryKeyString The primaryKeyString to set.
-   */
-  public void setPrimaryKeyString(String primaryKeyString) {
-    this.primaryKeyString = primaryKeyString;
-  }
-
-  /**
-   * @return the measureNamesString
-   */
-  public String getMeasureNamesString() {
-    return measureNamesString;
-  }
-
-  /**
-   * @param measureNamesString the measureNamesString to set
-   */
-  public void setMeasureNamesString(String measureNamesString) {
-    this.measureNamesString = measureNamesString;
-  }
-
-  /**
-   * @return Returns the aggType.
-   */
-  public String[] getAggType() {
-    return aggType;
-  }
-
-  /**
-   * @param aggType The aggType to set.
-   */
-  public void setAggType(String[] aggType) {
-    this.aggType = aggType;
-  }
-
-  /**
-   * @return Returns the actualDimensionColumns.
-   */
-  public String getActualDimensionColumns() {
-    return actualDimensionColumns;
-  }
-
-  /**
-   * @param actualDimensionColumns The actualDimensionColumns to set.
-   */
-  public void setActualDimensionColumns(String actualDimensionColumns) {
-    this.actualDimensionColumns = actualDimensionColumns;
-  }
-
-  /**
-   * getForgienKeyAndPrimaryKeyMapString
-   *
-   * @return String
-   */
-  public String getForgienKeyAndPrimaryKeyMapString() {
-    return forgienKeyAndPrimaryKeyMapString;
-  }
-
-  /**
-   * setForgienKeyAndPrimaryKeyMapString
-   *
-   * @param forgienKeyAndPrimaryKeyMapString void
-   */
-  public void setForgienKeyAndPrimaryKeyMapString(String forgienKeyAndPrimaryKeyMapString) {
-    this.forgienKeyAndPrimaryKeyMapString = forgienKeyAndPrimaryKeyMapString;
-  }
-
-  /**
-   * @return Returns the heirAndDimLens.
-   */
-  public String getHeirAndDimLens() {
-    return heirAndDimLens;
-  }
-
-  /**
-   * @param heirAndDimLens The heirAndDimLens to set.
-   */
-  public void setHeirAndDimLens(String heirAndDimLens) {
-    this.heirAndDimLens = heirAndDimLens;
-  }
-
-  /**
-   * @return Returns the measureDataTypeInfo.
-   */
-  public String getMeasureDataTypeInfo() {
-    return measureDataTypeInfo;
-  }
-
-  /**
-   * @param measureDataTypeInfo The measureDataTypeInfo to set.
-   */
-  public void setMeasureDataTypeInfo(String measureDataTypeInfo) {
-    this.measureDataTypeInfo = measureDataTypeInfo;
-  }
-
-  public String getColumnAndTableNameColumnMapForAgg() {
-    return columnAndTableNameColumnMapForAgg;
-  }
-
-  /**
-   * @return Returns the denormColumns.
-   */
-  public String getDenormColumns() {
-    return denormColumns;
-  }
-
-  /**
-   * @param denormColumns The denormColumns to set.
-   */
-  public void setDenormColumns(String denormColumns) {
-    this.denormColumns = denormColumns;
-  }
-
-  /**
-   * @return the aggClass
-   */
-  public String[] getAggClass() {
-    return aggClass;
-  }
-
-  /**
-   * @param measureUniqueColumnNamesString the measureUniqueColumnNamesString to set
-   */
-  public void setMeasureUniqueColumnNamesString(String measureUniqueColumnNamesString) {
-    this.measureUniqueColumnNamesString = measureUniqueColumnNamesString;
-  }
-
-  /**
-   * @param type the type to set
-   */
-  public void setType(char[] type) {
-    this.type = type;
-  }
-
-  public String getLevelAnddataType() {
-    return levelAnddataType;
-  }
-
-  public void setLevelAnddataType(String levelAnddataType) {
-    this.levelAnddataType = levelAnddataType;
-  }
-
-  /**
-   * getNoDictionaryDims.
-   *
-   * @return
-   */
-  public String getNoDictionaryDims() {
-    return noDictionaryDims;
-  }
-
-  /**
-   * setNoDictionaryDims.
-   *
-   * @param noDictionaryDims
-   */
-  public void setNoDictionaryDims(String noDictionaryDims) {
-    this.noDictionaryDims = noDictionaryDims;
-  }
-
-  /**
-   * Set Wrapper Object having the columnschemadetails
-   *
-   * @param columnSchemaDetailsWrapper
-   */
-  public void setColumnSchemaDetails(ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper) {
-    this.columnSchemaDetailsWrapper = columnSchemaDetailsWrapper;
-  }
-
-  /**
-   * return the Wrapper Object having the columnschemadetails
-   * @return
-   */
-  public ColumnSchemaDetailsWrapper getColumnSchemaDetails() {
-    return columnSchemaDetailsWrapper;
-  }
-
-  /**
-   * set wraper object having table options needed while dataload
-   *
-   * @return
-   */
-  public void setTableOptionWrapper(TableOptionWrapper tableOptionWrapper) {
-    this.tableOptionWrapper = tableOptionWrapper;
-  }
-
-  /**
-   * method returns the table options detail wrapper instance.
-   * @return
-   */
-  public TableOptionWrapper getTableOptionWrapper() {
-    return tableOptionWrapper;
-  }
-
-  public void setColumnGroupsString(String columnGroups) {
-    this.columnGroupsString = columnGroups;
-  }
-
-  /**
-   * @return columngroups
-   */
-  public String getColumnGroupsString() {
-    return columnGroupsString;
-  }
-
-  public Boolean[] getIsNoDictionaryDimMapping() {
-    return isNoDictionaryDimMapping;
-  }
-
-  public void setIsNoDictionaryDimMapping(Boolean[] isNoDictionaryDimMapping) {
-    this.isNoDictionaryDimMapping = isNoDictionaryDimMapping;
-  }
-
-  public void setColumnPropertiesString(String columnPropertiesString) {
-    this.columnPropertiesString = columnPropertiesString;
-  }
-
-  public String getColumnPropertiesString() {
-    return this.columnPropertiesString;
-  }
-
-  /**
-   * @return columngroups
-   */
-  public String getDimensionColumnsDataType() {
-    return columnsDataTypeString;
-  }
-
-  public void setDimensionColumnsDataType(String columnsDataTypeString) {
-    this.columnsDataTypeString = columnsDataTypeString;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
deleted file mode 100644
index 2f92699..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ /dev/null
@@ -1,531 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.mdkeygen;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.common.logging.impl.StandardLogService;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.datastore.block.SegmentProperties;
-import org.apache.carbondata.core.keygenerator.KeyGenException;
-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.CarbonMeasure;
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.core.util.DataTypeUtil;
-import org.apache.carbondata.core.util.path.CarbonStorePath;
-import org.apache.carbondata.core.util.path.CarbonTablePath;
-import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.mdkeygen.file.FileData;
-import org.apache.carbondata.processing.mdkeygen.file.FileManager;
-import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
-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.store.SingleThreadFinalSortFilesMerger;
-import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-import org.apache.carbondata.processing.util.NonDictionaryUtil;
-
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.row.RowMeta;
-import org.pentaho.di.core.row.ValueMeta;
-import org.pentaho.di.core.row.ValueMetaInterface;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStep;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-
-public class MDKeyGenStep extends BaseStep {
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(MDKeyGenStep.class.getName());
-
-  /**
-   * carbon mdkey generator step data class
-   */
-  private MDKeyGenStepData data;
-
-  /**
-   * carbon mdkey generator step meta
-   */
-  private MDKeyGenStepMeta meta;
-
-  /**
-   * dimension length
-   */
-  private int dimensionCount;
-
-  /**
-   * table name
-   */
-  private String tableName;
-
-  /**
-   * table blocksize in MB
-   */
-  private int blockSize;
-
-  /**
-   * File manager
-   */
-  private IFileManagerComposite fileManager;
-
-  private Map<Integer, GenericDataType> complexIndexMap;
-
-  private SegmentProperties segmentProperties;
-
-  private int[] colCardinality;
-
-  private List<ColumnSchema> wrapperColumnSchema;
-
-  /**
-   * readCounter
-   */
-  private long readCounter;
-
-  /**
-   * writeCounter
-   */
-  private long writeCounter;
-
-  private int measureCount;
-
-  private String dataFolderLocation;
-
-  private SingleThreadFinalSortFilesMerger finalMerger;
-
-  /**
-   * dataHandler
-   */
-  private CarbonFactHandler dataHandler;
-
-  private char[] aggType;
-
-  private String storeLocation;
-
-  private int[] dimLens;
-
-  /**
-   * to check whether dimension is of dictionary type
-   * or not
-   */
-  private boolean[] isNoDictionaryDimension;
-
-  /**
-   * to check whether dimension use inverted index
-   * or not
-   */
-  private boolean[] isUseInvertedIndex;
-
-  /**
-   * CarbonMDKeyGenStep
-   *
-   * @param stepMeta
-   * @param stepDataInterface
-   * @param copyNr
-   * @param transMeta
-   * @param trans
-   */
-  public MDKeyGenStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr,
-      TransMeta transMeta, Trans trans) {
-    super(stepMeta, stepDataInterface, copyNr, transMeta, trans);
-  }
-
-  /**
-   * Perform the equivalent of processing one row. Typically this means
-   * reading a row from input (getRow()) and passing a row to output
-   * (putRow)).
-   *
-   * @param smi The steps metadata to work with
-   * @param sdi The steps temporary working data to work with (database
-   *            connections, result sets, caches, temporary variables, etc.)
-   * @return false if no more rows can be processed or an error occurred.
-   * @throws KettleException
-   */
-  public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException {
-    meta = (MDKeyGenStepMeta) smi;
-    StandardLogService.setThreadName(meta.getPartitionID(), null);
-    data = (MDKeyGenStepData) sdi;
-
-    meta.initialize();
-    Object[] row = getRow();
-    if (first) {
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordMdkGenerateTotalTime(
-          meta.getPartitionID(), System.currentTimeMillis());
-      first = false;
-
-      data.outputRowMeta = new RowMeta();
-      boolean isExecutionRequired = setStepConfiguration();
-
-      if (!isExecutionRequired) {
-        processingComplete();
-        return false;
-      }
-      setStepOutputInterface();
-    }
-
-    if (null != row) {
-      putRow(data.outputRowMeta, new Object[measureCount + 1]);
-      return true;
-    }
-
-    try {
-      initDataHandler();
-      dataHandler.initialise();
-      finalMerger.startFinalMerge();
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordDictionaryValue2MdkAdd2FileTime(
-              meta.getPartitionID(), System.currentTimeMillis());
-      while (finalMerger.hasNext()) {
-        Object[] r = finalMerger.next();
-        readCounter++;
-        Object[] outputRow = process(r);
-        dataHandler.addDataToStore(outputRow);
-        writeCounter++;
-      }
-    } catch (CarbonDataWriterException e) {
-      LOGGER.error(e, "Failed for table: " + this.tableName + " in MDKeyGenStep");
-      throw new KettleException("Error while initializing data handler : " + e.getMessage());
-    } catch (Exception e) {
-      LOGGER.error(e, "Failed for table: " + this.tableName + " in MDKeyGenStep");
-      throw new KettleException("There is an unexpected error: " + e.getMessage());
-    } finally {
-      try {
-        dataHandler.finish();
-      } catch (CarbonDataWriterException e) {
-        LOGGER.error(e, "Failed for table: " + this.tableName + " in  finishing data handler");
-      } catch (Exception e) {
-        LOGGER.error(e, "Failed for table: " + this.tableName + " in  finishing data handler");
-      }
-    }
-    LOGGER.info("Record Procerssed For table: " + this.tableName);
-    String logMessage =
-        "Finished Carbon Mdkey Generation Step: Read: " + readCounter + ": Write: " + writeCounter;
-    LOGGER.info(logMessage);
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordTotalRecords(writeCounter);
-    processingComplete();
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordDictionaryValue2MdkAdd2FileTime(
-        meta.getPartitionID(), System.currentTimeMillis());
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordMdkGenerateTotalTime(
-        meta.getPartitionID(), System.currentTimeMillis());
-    return false;
-  }
-
-  private void processingComplete() throws KettleException {
-    if (null != dataHandler) {
-      try {
-        dataHandler.closeHandler();
-      } catch (CarbonDataWriterException e) {
-        LOGGER.error(e, e.getMessage());
-        throw new KettleException(e.getMessage());
-      } catch (Exception e) {
-        LOGGER.error(e, e.getMessage());
-        throw new KettleException("There is an unexpected error: " + e.getMessage());
-      }
-    }
-    setOutputDone();
-  }
-
-  /**
-   * This method will be used to get and update the step properties which will
-   * required to run this step
-   */
-  private boolean setStepConfiguration() {
-    this.tableName = meta.getTableName();
-    storeLocation = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(meta.getDatabaseName(), meta.getTableName(),
-            String.valueOf(meta.getTaskNo()), meta.getPartitionID(), meta.getSegmentId() + "",
-            false);
-    isNoDictionaryDimension =
-        NonDictionaryUtil.convertStringToBooleanArr(meta.getNoDictionaryDimsMapping());
-    isUseInvertedIndex =
-        NonDictionaryUtil.convertStringToBooleanArr(meta.getIsUseInvertedIndex());
-    fileManager = new FileManager();
-    fileManager.setName(CarbonCommonConstants.LOAD_FOLDER + meta.getSegmentId()
-        + CarbonCommonConstants.FILE_INPROGRESS_STATUS);
-
-    if (!(new File(storeLocation).exists())) {
-      LOGGER.error("Local data load folder location does not exist: " + storeLocation);
-      return false;
-    }
-
-    this.meta.setNoDictionaryCount(
-        NonDictionaryUtil.extractNoDictionaryCount(this.meta.getNoDictionaryDims()));
-
-    String levelCardinalityFilePath = storeLocation + File.separator +
-        CarbonCommonConstants.LEVEL_METADATA_FILE + meta.getTableName()
-        + CarbonCommonConstants.CARBON_METADATA_EXTENSION;
-    int[] dimLensWithComplex = null;
-    try {
-      dimLensWithComplex = CarbonUtil.getCardinalityFromLevelMetadataFile(levelCardinalityFilePath);
-    } catch (IOException e) {
-      LOGGER.error("Level cardinality file :: " + e.getMessage());
-      return false;
-    }
-    if (null == dimLensWithComplex) {
-      return false;
-    }
-    List<Integer> dimsLenList = new ArrayList<Integer>();
-    for (int eachDimLen : dimLensWithComplex) {
-      if (eachDimLen != 0) dimsLenList.add(eachDimLen);
-    }
-    dimLens = new int[dimsLenList.size()];
-    for (int i = 0; i < dimsLenList.size(); i++) {
-      dimLens[i] = dimsLenList.get(i);
-    }
-
-    this.dimensionCount = meta.getDimensionCount();
-
-    int simpleDimsCount = this.dimensionCount - meta.getComplexDimsCount();
-    int[] simpleDimsLen = new int[simpleDimsCount];
-    System.arraycopy(dimLens, 0, simpleDimsLen, 0, simpleDimsCount);
-
-    CarbonTable carbonTable = CarbonMetadata.getInstance()
-        .getCarbonTable(meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + tableName);
-    wrapperColumnSchema = CarbonUtil
-        .getColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
-            carbonTable.getMeasureByTableName(tableName));
-    blockSize = carbonTable.getBlockSizeInMB();
-    colCardinality =
-        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchema);
-    segmentProperties = new SegmentProperties(wrapperColumnSchema, colCardinality);
-    // Actual primitive dimension used to generate start & end key
-
-    data.generator = segmentProperties.getDimensionKeyGenerator();
-
-    //To Set MDKey Index of each primitive type in complex type
-    int surrIndex = simpleDimsCount;
-    Iterator<Entry<String, GenericDataType>> complexMap =
-        meta.getComplexTypes().entrySet().iterator();
-    complexIndexMap = new HashMap<Integer, GenericDataType>(meta.getComplexDimsCount());
-    while (complexMap.hasNext()) {
-      Entry<String, GenericDataType> complexDataType = complexMap.next();
-      complexDataType.getValue().setOutputArrayIndex(0);
-      complexIndexMap.put(simpleDimsCount, complexDataType.getValue());
-      simpleDimsCount++;
-      List<GenericDataType> primitiveTypes = new ArrayList<GenericDataType>();
-      complexDataType.getValue().getAllPrimitiveChildren(primitiveTypes);
-      for (GenericDataType eachPrimitive : primitiveTypes) {
-        eachPrimitive.setSurrogateIndex(surrIndex++);
-      }
-    }
-
-    this.measureCount = meta.getMeasureCount();
-
-    String metaDataFileName = CarbonCommonConstants.MEASURE_METADATA_FILE_NAME + this.tableName
-        + CarbonCommonConstants.MEASUREMETADATA_FILE_EXT
-        + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-
-    FileData fileData = new FileData(metaDataFileName, storeLocation);
-    fileManager.add(fileData);
-    // Set the data file location
-    this.dataFolderLocation =
-        storeLocation + File.separator + CarbonCommonConstants.SORT_TEMP_FILE_LOCATION;
-    return true;
-  }
-
-  private void initDataHandler() {
-    int simpleDimsCount = this.dimensionCount - meta.getComplexDimsCount();
-    int[] simpleDimsLen = new int[simpleDimsCount];
-    System.arraycopy(dimLens, 0, simpleDimsLen, 0, simpleDimsCount);
-    CarbonDataFileAttributes carbonDataFileAttributes =
-        new CarbonDataFileAttributes(meta.getTaskNo(), meta.getFactTimeStamp());
-    initAggType();
-    String carbonDataDirectoryPath = getCarbonDataFolderLocation();
-    finalMerger = new SingleThreadFinalSortFilesMerger(dataFolderLocation, tableName,
-        dimensionCount - meta.getComplexDimsCount(), meta.getComplexDimsCount(), measureCount,
-        meta.getNoDictionaryCount(), aggType, isNoDictionaryDimension, true);
-    CarbonFactDataHandlerModel carbonFactDataHandlerModel = getCarbonFactDataHandlerModel();
-    carbonFactDataHandlerModel.setPrimitiveDimLens(simpleDimsLen);
-    carbonFactDataHandlerModel.setCarbonDataFileAttributes(carbonDataFileAttributes);
-    carbonFactDataHandlerModel.setCarbonDataDirectoryPath(carbonDataDirectoryPath);
-    carbonFactDataHandlerModel.setIsUseInvertedIndex(isUseInvertedIndex);
-    if (meta.getNoDictionaryCount() > 0 || meta.getComplexDimsCount() > 0) {
-      carbonFactDataHandlerModel.setMdKeyIndex(measureCount + 1);
-    } else {
-      carbonFactDataHandlerModel.setMdKeyIndex(measureCount);
-    }
-    dataHandler = new CarbonFactDataHandlerColumnar(carbonFactDataHandlerModel);
-  }
-
-  /**
-   * This method will create a model object for carbon fact data handler
-   *
-   * @return
-   */
-  private CarbonFactDataHandlerModel getCarbonFactDataHandlerModel() {
-    CarbonFactDataHandlerModel carbonFactDataHandlerModel = new CarbonFactDataHandlerModel();
-    carbonFactDataHandlerModel.setDatabaseName(meta.getDatabaseName());
-    carbonFactDataHandlerModel.setTableName(tableName);
-    carbonFactDataHandlerModel.setMeasureCount(measureCount);
-    carbonFactDataHandlerModel.setMdKeyLength(data.generator.getKeySizeInBytes());
-    carbonFactDataHandlerModel.setStoreLocation(storeLocation);
-    carbonFactDataHandlerModel.setDimLens(dimLens);
-    carbonFactDataHandlerModel.setNoDictionaryCount(meta.getNoDictionaryCount());
-    carbonFactDataHandlerModel.setDimensionCount(dimensionCount);
-    carbonFactDataHandlerModel.setComplexIndexMap(complexIndexMap);
-    carbonFactDataHandlerModel.setSegmentProperties(segmentProperties);
-    carbonFactDataHandlerModel.setColCardinality(colCardinality);
-    carbonFactDataHandlerModel.setDataWritingRequest(true);
-    carbonFactDataHandlerModel.setAggType(aggType);
-    carbonFactDataHandlerModel.setFactDimLens(dimLens);
-    carbonFactDataHandlerModel.setWrapperColumnSchema(wrapperColumnSchema);
-    carbonFactDataHandlerModel.setBlockSizeInMB(blockSize);
-    return carbonFactDataHandlerModel;
-  }
-
-  private void initAggType() {
-    aggType = new char[measureCount];
-    Arrays.fill(aggType, 'n');
-    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(
-        meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName());
-    List<CarbonMeasure> measures = carbonTable.getMeasureByTableName(meta.getTableName());
-    for (int i = 0; i < measureCount; i++) {
-      aggType[i] = DataTypeUtil.getAggType(measures.get(i).getDataType());
-    }
-  }
-
-  /**
-   * This method will be used for setting the output interface.
-   * Output interface is how this step will process the row to next step
-   */
-  private void setStepOutputInterface() {
-    ValueMetaInterface[] out = new ValueMetaInterface[measureCount + 1];
-
-    for (int i = 0; i < measureCount; i++) {
-      out[i] = new ValueMeta("measure" + i, ValueMetaInterface.TYPE_NUMBER,
-          ValueMetaInterface.STORAGE_TYPE_NORMAL);
-      out[i].setStorageMetadata(new ValueMeta("measure" + i, ValueMetaInterface.TYPE_NUMBER,
-          ValueMetaInterface.STORAGE_TYPE_NORMAL));
-    }
-
-    out[out.length - 1] = new ValueMeta("id", ValueMetaInterface.TYPE_BINARY,
-        ValueMetaInterface.STORAGE_TYPE_BINARY_STRING);
-    out[out.length - 1].setStorageMetadata(new ValueMeta("id", ValueMetaInterface.TYPE_STRING,
-        ValueMetaInterface.STORAGE_TYPE_NORMAL));
-    out[out.length - 1].setLength(256);
-    out[out.length - 1].setStringEncoding(CarbonCommonConstants.BYTE_ENCODING);
-    out[out.length - 1].getStorageMetadata().setStringEncoding(CarbonCommonConstants.BYTE_ENCODING);
-
-    data.outputRowMeta.setValueMetaList(Arrays.asList(out));
-  }
-
-  /**
-   * This method will be used to get the row from previous step and then it
-   * will generate the mdkey and then send the mdkey to next step
-   *
-   * @param row input row
-   * @throws KettleException
-   */
-  private Object[] process(Object[] row) throws KettleException {
-    Object[] outputRow = null;
-    // adding one for the high cardinality dims byte array.
-    if (meta.getNoDictionaryCount() > 0 || meta.getComplexDimsCount() > 0) {
-      outputRow = new Object[measureCount + 1 + 1];
-    } else {
-      outputRow = new Object[measureCount + 1];
-    }
-
-    int l = 0;
-    int index = 0;
-    for (int i = 0; i < measureCount; i++) {
-      if (aggType[i] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) {
-        outputRow[l++] = NonDictionaryUtil.getMeasure(index++, row);
-      } else if (aggType[i] == CarbonCommonConstants.BIG_INT_MEASURE) {
-        outputRow[l++] = (Long) NonDictionaryUtil.getMeasure(index++, row);
-      } else {
-        outputRow[l++] = (Double) NonDictionaryUtil.getMeasure(index++, row);
-      }
-    }
-    outputRow[l] = NonDictionaryUtil.getByteArrayForNoDictionaryCols(row);
-
-    int[] highCardExcludedRows = new int[segmentProperties.getDimColumnsCardinality().length];
-    for (int i = 0; i < highCardExcludedRows.length; i++) {
-      Object key = NonDictionaryUtil.getDimension(i, row);
-      highCardExcludedRows[i] = (Integer) key;
-    }
-    try {
-      outputRow[outputRow.length - 1] = data.generator.generateKey(highCardExcludedRows);
-    } catch (KeyGenException e) {
-      throw new KettleException("unable to generate the mdkey", e);
-    }
-
-    return outputRow;
-  }
-
-  /**
-   * This method will get the store location for the given path, segment id and partition id
-   *
-   * @return data directory path
-   */
-  private String getCarbonDataFolderLocation() {
-    String carbonStorePath =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS);
-    CarbonTable carbonTable = CarbonMetadata.getInstance().getCarbonTable(
-        meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + meta.getTableName());
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTable.getCarbonTableIdentifier());
-    String carbonDataDirectoryPath =
-        carbonTablePath.getCarbonDataDirectoryPath(meta.getPartitionID(), meta.getSegmentId() + "");
-    return carbonDataDirectoryPath;
-  }
-
-  /**
-   * Initialize and do work where other steps need to wait for...
-   *
-   * @param smi The metadata to work with
-   * @param sdi The data to initialize
-   * @return step initialize or not
-   */
-  public boolean init(StepMetaInterface smi, StepDataInterface sdi) {
-    meta = (MDKeyGenStepMeta) smi;
-    data = (MDKeyGenStepData) sdi;
-
-    return super.init(smi, sdi);
-  }
-
-  /**
-   * Dispose of this step: close files, empty logs, etc.
-   *
-   * @param smi The metadata to work with
-   * @param sdi The data to dispose of
-   */
-  public void dispose(StepMetaInterface smi, StepDataInterface sdi) {
-    meta = (MDKeyGenStepMeta) smi;
-    data = (MDKeyGenStepData) sdi;
-    super.dispose(smi, sdi);
-    dataHandler = null;
-    finalMerger = null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepData.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepData.java b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepData.java
deleted file mode 100644
index b2c682c..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepData.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.mdkeygen;
-
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.trans.step.BaseStepData;
-import org.pentaho.di.trans.step.StepDataInterface;
-
-public class MDKeyGenStepData extends BaseStepData implements StepDataInterface {
-  /**
-   * outputRowMeta
-   */
-  protected RowMetaInterface outputRowMeta;
-
-  /**
-   * generator for each column independently
-   */
-  protected KeyGenerator generator;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index c1be2ef..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStepMeta.java
+++ /dev/null
@@ -1,529 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.mdkeygen;
-
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-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.util.CarbonDataProcessorUtil;
-
-import org.pentaho.di.core.CheckResultInterface;
-import org.pentaho.di.core.Counter;
-import org.pentaho.di.core.database.DatabaseMeta;
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.exception.KettleXMLException;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.core.xml.XMLHandler;
-import org.pentaho.di.i18n.BaseMessages;
-import org.pentaho.di.repository.ObjectId;
-import org.pentaho.di.repository.Repository;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStepMeta;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-import org.w3c.dom.Node;
-
-public class MDKeyGenStepMeta extends BaseStepMeta implements StepMetaInterface {
-  /**
-   * for i18n purposes
-   */
-  private static Class<?> pkg = MDKeyGenStepMeta.class;
-
-  /**
-   * tableName
-   */
-  private String tableName;
-
-  /**
-   * numberOfCores
-   */
-  private String numberOfCores;
-
-  /**
-   * databaseName
-   */
-  private String databaseName;
-
-  /**
-   * aggregateLevels
-   */
-  private String aggregateLevels;
-
-  /**
-   * measureCount
-   */
-  private String measureCount;
-
-  /**
-   * dimensionCount
-   */
-  private String dimensionCount;
-
-  /**
-   * complexDimsCount
-   */
-  private String complexDimsCount;
-
-  /**
-   * ComplexTypeString
-   */
-  private String complexTypeString;
-
-  private Map<String, GenericDataType> complexTypes;
-
-  /**
-   * It is column groups in below format
-   * 0,1~2~3,4,5,6~7~8,9
-   * groups are
-   * ,-> all ordinal with different group id
-   * ~-> all ordinal with same group id
-   */
-  private String columnGroupsString;
-  private String noDictionaryDims;
-
-  /**
-   * noDictionaryCount
-   */
-  private int noDictionaryCount;
-
-  private String measureDataType;
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-  /**
-   * new load start time
-   */
-  private String factTimeStamp;
-  /**
-   * partitionID
-   */
-  private String partitionID;
-  /**
-   * Id of the load folder
-   */
-  private String segmentId;
-  /**
-   * To determine the column whether is dictionary or not.
-   */
-  private String noDictionaryDimsMapping;
-  /**
-   * To determine the column whether use inverted index or not.
-   */
-  private String isUseInvertedIndex;
-
-  /**
-   * Constructor
-   */
-  public MDKeyGenStepMeta() {
-    super();
-  }
-
-  @Override public void setDefault() {
-    tableName = "";
-    numberOfCores = "";
-    aggregateLevels = "";
-    tableName = "";
-    databaseName = "";
-    columnGroupsString = "";
-    noDictionaryDims = "";
-    measureDataType = "";
-    taskNo = "";
-    factTimeStamp = "";
-    partitionID = "";
-    segmentId = "";
-    noDictionaryDimsMapping = "";
-  }
-
-  public String getXML() {
-    StringBuilder retval = new StringBuilder(150);
-
-    retval.append("    ").append(XMLHandler.addTagValue("TableName", tableName));
-    retval.append("    ").append(XMLHandler.addTagValue("AggregateLevels", aggregateLevels));
-    retval.append("    ").append(XMLHandler.addTagValue("NumberOfCores", numberOfCores));
-    retval.append("    ").append(XMLHandler.addTagValue("tableName", tableName));
-    retval.append("    ").append(XMLHandler.addTagValue("databaseName", databaseName));
-    retval.append("    ").append(XMLHandler.addTagValue("noDictionaryDims", noDictionaryDims));
-    retval.append("    ").append(XMLHandler.addTagValue("measureCount", measureCount));
-    retval.append("    ").append(XMLHandler.addTagValue("dimensionsStoreType", columnGroupsString));
-    retval.append("    ").append(XMLHandler.addTagValue("dimensionCount", dimensionCount));
-    retval.append("    ").append(XMLHandler.addTagValue("complexDimsCount", complexDimsCount));
-    retval.append("    ").append(XMLHandler.addTagValue("complexTypeString", complexTypeString));
-    retval.append("    ").append(XMLHandler.addTagValue("measureDataType", measureDataType));
-    retval.append("    ").append(XMLHandler.addTagValue("taskNo", taskNo));
-    retval.append("    ").append(XMLHandler.addTagValue("factTimeStamp", factTimeStamp));
-    retval.append("    ").append(XMLHandler.addTagValue("factTimeStamp", factTimeStamp));
-    retval.append("    ").append(XMLHandler.addTagValue("partitionID", partitionID));
-    retval.append("    ").append(XMLHandler.addTagValue("isUseInvertedIndex", isUseInvertedIndex));
-    retval.append("    ").append(XMLHandler.addTagValue("segmentId", segmentId));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("noDictionaryDimsMapping", noDictionaryDimsMapping));
-    return retval.toString();
-  }
-
-  @Override
-  public void loadXML(Node stepnode, List<DatabaseMeta> databases, Map<String, Counter> counters)
-      throws KettleXMLException {
-    try {
-      tableName = XMLHandler.getTagValue(stepnode, "TableName");
-      aggregateLevels = XMLHandler.getTagValue(stepnode, "AggregateLevels");
-      numberOfCores = XMLHandler.getTagValue(stepnode, "NumberOfCores");
-      databaseName = XMLHandler.getTagValue(stepnode, "databaseName");
-      tableName = XMLHandler.getTagValue(stepnode, "tableName");
-      noDictionaryDims = XMLHandler.getTagValue(stepnode, "noDictionaryDims");
-      measureCount = XMLHandler.getTagValue(stepnode, "measureCount");
-      columnGroupsString = XMLHandler.getTagValue(stepnode, "dimensionsStoreType");
-      dimensionCount = XMLHandler.getTagValue(stepnode, "dimensionCount");
-      complexDimsCount = XMLHandler.getTagValue(stepnode, "complexDimsCount");
-      complexTypeString = XMLHandler.getTagValue(stepnode, "complexTypeString");
-      measureDataType = XMLHandler.getTagValue(stepnode, "measureDataType");
-      taskNo = XMLHandler.getTagValue(stepnode, "taskNo");
-      factTimeStamp = XMLHandler.getTagValue(stepnode, "factTimeStamp");
-      partitionID = XMLHandler.getTagValue(stepnode, "partitionID");
-      isUseInvertedIndex = XMLHandler.getTagValue(stepnode, "isUseInvertedIndex");
-      segmentId = XMLHandler.getTagValue(stepnode, "segmentId");
-      noDictionaryDimsMapping = XMLHandler.getTagValue(stepnode, "noDictionaryDimsMapping");
-    } catch (Exception e) {
-      throw new KettleXMLException("Unable to read step info from XML node", e);
-    }
-  }
-
-  @Override public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep)
-      throws KettleException {
-    try {
-      rep.saveStepAttribute(idTransformation, idStep, "TableName", tableName);
-      rep.saveStepAttribute(idTransformation, idStep, "AggregateLevels", aggregateLevels);
-      rep.saveStepAttribute(idTransformation, idStep, "NumberOfCores", numberOfCores);
-      rep.saveStepAttribute(idTransformation, idStep, "databaseName", databaseName);
-      rep.saveStepAttribute(idTransformation, idStep, "tableName", tableName);
-      rep.saveStepAttribute(idTransformation, idStep, "noDictionaryDims", noDictionaryDims);
-      rep.saveStepAttribute(idTransformation, idStep, "measureCount", measureCount);
-      rep.saveStepAttribute(idTransformation, idStep, "dimensionsStoreType", columnGroupsString);
-      rep.saveStepAttribute(idTransformation, idStep, "dimensionCount", dimensionCount);
-      rep.saveStepAttribute(idTransformation, idStep, "complexDimsCount", complexDimsCount);
-      rep.saveStepAttribute(idTransformation, idStep, "complexTypeString", complexTypeString);
-      rep.saveStepAttribute(idTransformation, idStep, "measureDataType", measureDataType);
-      rep.saveStepAttribute(idTransformation, idStep, "taskNo", taskNo);
-      rep.saveStepAttribute(idTransformation, idStep, "factTimeStamp", factTimeStamp);
-      rep.saveStepAttribute(idTransformation, idStep, "partitionID", partitionID);
-      rep.saveStepAttribute(idTransformation, idStep, "isUseInvertedIndex", isUseInvertedIndex);
-      rep.saveStepAttribute(idTransformation, idStep, "segmentId", segmentId);
-      rep.saveStepAttribute(idTransformation, idStep, "noDictionaryDimsMapping",
-          noDictionaryDimsMapping);
-    } catch (Exception e) {
-      throw new KettleException(
-          BaseMessages.getString(pkg, "TemplateStep.Exception.UnableToSaveStepInfoToRepository")
-              + idStep, e);
-    }
-
-  }
-
-  @Override public void readRep(Repository rep, ObjectId idStep, List<DatabaseMeta> databases,
-      Map<String, Counter> counters) throws KettleException {
-    try {
-      tableName = rep.getStepAttributeString(idStep, "TableName");
-      aggregateLevels = rep.getStepAttributeString(idStep, "AggregateLevels");
-      numberOfCores = rep.getStepAttributeString(idStep, "NumberOfCores");
-      databaseName = rep.getStepAttributeString(idStep, "databaseName");
-      tableName = rep.getStepAttributeString(idStep, "tableName");
-      noDictionaryDims = rep.getStepAttributeString(idStep, "noDictionaryDims");
-      measureCount = rep.getStepAttributeString(idStep, "measureCount");
-      columnGroupsString = rep.getStepAttributeString(idStep, "dimensionsStoreType");
-      dimensionCount = rep.getStepAttributeString(idStep, "dimensionCount");
-      complexDimsCount = rep.getStepAttributeString(idStep, "complexDimsCount");
-      complexTypeString = rep.getStepAttributeString(idStep, "complexTypeString");
-      measureDataType = rep.getStepAttributeString(idStep, "measureDataType");
-      taskNo = rep.getStepAttributeString(idStep, "taskNo");
-      factTimeStamp = rep.getStepAttributeString(idStep, "factTimeStamp");
-      partitionID = rep.getStepAttributeString(idStep, "partitionID");
-      isUseInvertedIndex = rep.getStepAttributeString(idStep, "isUseInvertedIndex");
-      segmentId = rep.getStepAttributeString(idStep, "segmentId");
-      noDictionaryDimsMapping = rep.getStepAttributeString(idStep, "noDictionaryDimsMapping");
-    } catch (Exception e) {
-      throw new KettleException(BaseMessages
-          .getString(pkg, "CarbonMDKeyStepMeta.Exception.UnexpectedErrorInReadingStepInfo"), e);
-    }
-  }
-
-  @Override
-  public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr,
-      TransMeta transMeta, Trans trans) {
-    return new MDKeyGenStep(stepMeta, stepDataInterface, copyNr, transMeta, trans);
-  }
-
-  @Override
-  public void check(List<CheckResultInterface> remarks, TransMeta transMeta, StepMeta stepMeta,
-      RowMetaInterface prev, String[] input, String[] output, RowMetaInterface info) {
-    CarbonDataProcessorUtil.checkResult(remarks, stepMeta, input);
-  }
-
-  @Override public StepDataInterface getStepData() {
-    return new MDKeyGenStepData();
-  }
-
-  public String getTableName() {
-    return tableName;
-  }
-
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  public void setAggregateLevels(String aggregateLevels) {
-    this.aggregateLevels = aggregateLevels;
-  }
-
-  public Map<String, GenericDataType> getComplexTypes() {
-    return complexTypes;
-  }
-
-  public void setNumberOfCores(String numberOfCores) {
-    this.numberOfCores = numberOfCores;
-  }
-
-  /**
-   * @return the databaseName
-   */
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  /**
-   * @param databaseName the databaseName to set
-   */
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * @return the measureCount
-   */
-  public int getMeasureCount() {
-    return Integer.parseInt(measureCount);
-  }
-
-  /**
-   * @param measureCount the measureCount to set
-   */
-  public void setMeasureCount(String measureCount) {
-    this.measureCount = measureCount;
-  }
-
-  /**
-   * @return the dimensionCount
-   */
-  public int getDimensionCount() {
-    return Integer.parseInt(dimensionCount);
-  }
-
-  /**
-   * @param dimensionCount the dimensionCount to set
-   */
-  public void setDimensionCount(String dimensionCount) {
-    this.dimensionCount = dimensionCount;
-  }
-
-  /**
-   * @return the complexDimsCount
-   */
-  public int getComplexDimsCount() {
-    return Integer.parseInt(complexDimsCount);
-  }
-
-  /**
-   * @param complexDimsCount the complexDimsCount to set
-   */
-  public void setComplexDimsCount(String complexDimsCount) {
-    this.complexDimsCount = complexDimsCount;
-  }
-
-  /**
-   * @param complexTypeString the complexTypeString to set
-   */
-  public void setComplexTypeString(String complexTypeString) {
-    this.complexTypeString = complexTypeString;
-  }
-
-  /**
-   * @return
-   */
-  public String getNoDictionaryDims() {
-    return noDictionaryDims;
-  }
-
-  /**
-   * @param noDictionaryDims
-   */
-  public void setNoDictionaryDims(String noDictionaryDims) {
-    this.noDictionaryDims = noDictionaryDims;
-  }
-
-  /**
-   * @return the noDictionaryCount
-   */
-  public int getNoDictionaryCount() {
-    return noDictionaryCount;
-  }
-
-  /**
-   * @param noDictionaryCount the noDictionaryCount to set
-   */
-  public void setNoDictionaryCount(int noDictionaryCount) {
-    this.noDictionaryCount = noDictionaryCount;
-  }
-
-  public void setColumnGroupsString(String columnGroups) {
-    this.columnGroupsString = columnGroups;
-
-  }
-
-  public void initialize() {
-    complexTypes = getComplexTypesMap(complexTypeString);
-  }
-
-  private Map<String, GenericDataType> getComplexTypesMap(String complexTypeString) {
-    if (null == complexTypeString) {
-      return new LinkedHashMap<>();
-    }
-    Map<String, GenericDataType> complexTypesMap = new LinkedHashMap<String, GenericDataType>();
-    String[] hierarchies = complexTypeString.split(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER);
-    for (int i = 0; i < hierarchies.length; i++) {
-      String[] levels = hierarchies[i].split(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      String[] levelInfo = levels[0].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      GenericDataType g = levelInfo[1].equals(CarbonCommonConstants.ARRAY) ?
-          new ArrayDataType(levelInfo[0], "", levelInfo[3]) :
-          new StructDataType(levelInfo[0], "", levelInfo[3]);
-      complexTypesMap.put(levelInfo[0], g);
-      for (int j = 1; j < levels.length; j++) {
-        levelInfo = levels[j].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-        switch (levelInfo[1]) {
-          case CarbonCommonConstants.ARRAY:
-            g.addChildren(new ArrayDataType(levelInfo[0], levelInfo[2], levelInfo[3]));
-            break;
-          case CarbonCommonConstants.STRUCT:
-            g.addChildren(new StructDataType(levelInfo[0], levelInfo[2], levelInfo[3]));
-            break;
-          default:
-            g.addChildren(new PrimitiveDataType(levelInfo[0], levelInfo[2], levelInfo[3],
-                Integer.parseInt(levelInfo[4])));
-        }
-      }
-    }
-    return complexTypesMap;
-  }
-
-  public void setMeasureDataType(String measureDataType) {
-    this.measureDataType = measureDataType;
-  }
-
-  /**
-   * @return
-   */
-  public int getTaskNo() {
-    return Integer.parseInt(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;
-  }
-
-  /**
-   * @return partitionId
-   */
-  public String getPartitionID() {
-    return partitionID;
-  }
-
-  /**
-   * @param partitionID
-   */
-  public void setPartitionID(String partitionID) {
-    this.partitionID = partitionID;
-  }
-
-  /**
-   * return segmentId
-   *
-   * @return
-   */
-  public String getSegmentId() {
-    return segmentId;
-  }
-
-  /**
-   * set segment Id
-   *
-   * @param segmentId
-   */
-  public void setSegmentId(String segmentId) {
-    this.segmentId = segmentId;
-  }
-
-  /**
-   * @return the noDictionaryDimsMapping
-   */
-  public String getNoDictionaryDimsMapping() {
-    return noDictionaryDimsMapping;
-  }
-
-  /**
-   * @param noDictionaryDimsMapping the noDictionaryDimsMapping to set
-   */
-  public void setNoDictionaryDimsMapping(String noDictionaryDimsMapping) {
-    this.noDictionaryDimsMapping = noDictionaryDimsMapping;
-  }
-  /**
-   * @return isUseInvertedIndex
-   */
-  public String getIsUseInvertedIndex() {
-    return isUseInvertedIndex;
-  }
-
-  /**
-   * @param isUseInvertedIndex the bool array whether use inverted index to set
-   */
-  public void setIsUseInvertedIndex(String isUseInvertedIndex) {
-    this.isUseInvertedIndex = isUseInvertedIndex;
-  }
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/messages/messages_en_US.properties
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/messages/messages_en_US.properties b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/messages/messages_en_US.properties
deleted file mode 100644
index 3608e57..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/messages/messages_en_US.properties
+++ /dev/null
@@ -1,22 +0,0 @@
-#
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-#
-CarbonMDkeyDialog.Shell.Title=Carbon MDkey Generator Step
-CarbonMDkeyDialog.TableName.Label=Table Name
-CarbonMDkeyDialog.SchemaFieldName.Label=Aggregate Levels
-CarbonMDkeyDialog.StoreLocation.Label=Store Location
-CarbonMDkeyDialog.NumberOfCores.Label=Number Of Cores
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStep.java b/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStep.java
deleted file mode 100644
index 4562793..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStep.java
+++ /dev/null
@@ -1,159 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.step;
-
-import java.io.File;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.common.logging.impl.StandardLogService;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStep;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-
-public class CarbonSliceMergerStep extends BaseStep {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonSliceMergerStep.class.getName());
-  /**
-   * carbon data writer step data class
-   */
-  private CarbonSliceMergerStepData data;
-
-  /**
-   * carbon data writer step meta
-   */
-  private CarbonSliceMergerStepMeta meta;
-
-  /**
-   * readCounter
-   */
-  private long readCounter;
-
-  /**
-   * writeCounter
-   */
-  private long writeCounter;
-
-  /**
-   * CarbonSliceMergerStep Constructor
-   *
-   * @param stepMeta          stepMeta
-   * @param stepDataInterface stepDataInterface
-   * @param copyNr            copyNr
-   * @param transMeta         transMeta
-   * @param trans             trans
-   */
-  public CarbonSliceMergerStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr,
-      TransMeta transMeta, Trans trans) {
-    super(stepMeta, stepDataInterface, copyNr, transMeta, trans);
-  }
-
-  /**
-   * Perform the equivalent of processing one row. Typically this means
-   * reading a row from input (getRow()) and passing a row to output
-   * (putRow)).
-   *
-   * @param smi The steps metadata to work with
-   * @param sdi The steps temporary working data to work with (database
-   *            connections, result sets, caches, temporary variables, etc.)
-   * @return false if no more rows can be processed or an error occurred.
-   * @throws KettleException
-   */
-  public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException {
-    try {
-      // carbon data writer step meta
-      meta = (CarbonSliceMergerStepMeta) smi;
-      StandardLogService.setThreadName(StandardLogService.getPartitionID(meta.getTableName()),
-          null);
-      // carbon data writer step data
-      data = (CarbonSliceMergerStepData) sdi;
-
-      // get row from previous step, blocks when needed!
-      Object[] row = getRow();
-      // if row is null then there is no more incoming data
-      if (null == row) {
-        renameFolders();
-
-        LOGGER.info("Record Procerssed For table: " + meta.getTabelName());
-        String logMessage =
-            "Summary: Carbon Slice Merger Step: Read: " + readCounter + ": Write: " + writeCounter;
-        LOGGER.info(logMessage);
-        // step processing is finished
-        setOutputDone();
-        // return false
-        return false;
-      }
-
-      if (first) {
-        first = false;
-        if (getInputRowMeta() != null) {
-          this.data.setOutputRowMeta((RowMetaInterface) getInputRowMeta().clone());
-          this.meta.getFields(data.getOutputRowMeta(), getStepname(), null, null, this);
-        }
-      }
-      readCounter++;
-    } catch (Exception ex) {
-      LOGGER.error(ex);
-      throw new RuntimeException(ex);
-    }
-    return true;
-  }
-
-  private void renameFolders() {
-    CarbonDataProcessorUtil.renameBadRecordsFromInProgressToNormal(
-        meta.getDatabaseName() + File.separator + meta.getTableName() + File.separator + meta
-            .getTaskNo());
-  }
-
-  /**
-   * Initialize and do work where other steps need to wait for...
-   *
-   * @param smi The metadata to work with
-   * @param sdi The data to initialize
-   * @return step initialize or not
-   */
-  public boolean init(StepMetaInterface smi, StepDataInterface sdi) {
-    meta = (CarbonSliceMergerStepMeta) smi;
-    data = (CarbonSliceMergerStepData) sdi;
-    return super.init(smi, sdi);
-  }
-
-  /**
-   * Dispose of this step: close files, empty logs, etc.
-   *
-   * @param smi The metadata to work with
-   * @param sdi The data to dispose of
-   */
-  public void dispose(StepMetaInterface smi, StepDataInterface sdi) {
-    meta = (CarbonSliceMergerStepMeta) smi;
-    data = (CarbonSliceMergerStepData) sdi;
-    super.dispose(smi, sdi);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepData.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepData.java b/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepData.java
deleted file mode 100644
index b989046..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/step/CarbonSliceMergerStepData.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.merger.step;
-
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.trans.step.BaseStepData;
-import org.pentaho.di.trans.step.StepDataInterface;
-
-public class CarbonSliceMergerStepData extends BaseStepData implements StepDataInterface {
-  /**
-   * outputRowMeta
-   */
-  private RowMetaInterface outputRowMeta;
-
-  public CarbonSliceMergerStepData() {
-    super();
-  }
-
-  public RowMetaInterface getOutputRowMeta() {
-    return outputRowMeta;
-  }
-
-  public void setOutputRowMeta(RowMetaInterface outputRowMeta) {
-    this.outputRowMeta = outputRowMeta;
-  }
-}


[02/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
deleted file mode 100644
index eb0f52b..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
+++ /dev/null
@@ -1,339 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-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.keygenerator.KeyGenException;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-import org.apache.carbondata.core.metadata.CarbonMetadata;
-import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
-import org.apache.carbondata.core.metadata.ColumnIdentifier;
-import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-import org.apache.carbondata.core.writer.ByteArrayHolder;
-import org.apache.carbondata.core.writer.HierarchyValueWriterForCSV;
-import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.mdkeygen.file.FileData;
-import org.apache.carbondata.processing.mdkeygen.file.FileManager;
-import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
-import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetails;
-import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper;
-import org.apache.carbondata.processing.schema.metadata.ColumnsInfo;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import org.pentaho.di.core.exception.KettleException;
-
-public class FileStoreSurrogateKeyGenForCSV extends CarbonCSVBasedDimSurrogateKeyGen {
-
-  /**
-   * hierValueWriter
-   */
-  private Map<String, HierarchyValueWriterForCSV> hierValueWriter;
-
-  /**
-   * keyGenerator
-   */
-  private Map<String, KeyGenerator> keyGenerator;
-
-  /**
-   * LOAD_FOLDER
-   */
-  private String loadFolderName;
-
-  /**
-   * primaryKeyStringArray
-   */
-  private String[] primaryKeyStringArray;
-  /**
-   * partitionID
-   */
-  private String partitionID;
-  /**
-   * load Id
-   */
-  private String segmentId;
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-
-  /**
-   * @param columnsInfo
-   * @throws IOException
-   */
-  public FileStoreSurrogateKeyGenForCSV(ColumnsInfo columnsInfo, String partitionID,
-      String segmentId, String taskNo) throws IOException {
-    super(columnsInfo);
-    populatePrimaryKeyarray(dimInsertFileNames, columnsInfo.getPrimaryKeyMap());
-    this.partitionID = partitionID;
-    this.segmentId = segmentId;
-    this.taskNo = taskNo;
-    keyGenerator = new HashMap<String, KeyGenerator>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    setStoreFolderWithLoadNumber(
-        checkAndCreateLoadFolderNumber(columnsInfo.getDatabaseName(),
-            columnsInfo.getTableName()));
-    fileManager = new FileManager();
-    fileManager.setName(loadFolderName + CarbonCommonConstants.FILE_INPROGRESS_STATUS);
-
-    hierValueWriter = new HashMap<String, HierarchyValueWriterForCSV>(
-        CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    for (Entry<String, String> entry : hierInsertFileNames.entrySet()) {
-      String hierFileName = entry.getValue().trim();
-      hierValueWriter.put(entry.getKey(),
-          new HierarchyValueWriterForCSV(hierFileName, getStoreFolderWithLoadNumber()));
-      Map<String, KeyGenerator> keyGenerators = columnsInfo.getKeyGenerators();
-      keyGenerator.put(entry.getKey(), keyGenerators.get(entry.getKey()));
-      FileData fileData = new FileData(hierFileName, getStoreFolderWithLoadNumber());
-      fileData.setHierarchyValueWriter(hierValueWriter.get(entry.getKey()));
-      fileManager.add(fileData);
-    }
-    populateCache();
-    //Update the primary key surroagate key map
-    updatePrimaryKeyMaxSurrogateMap();
-  }
-
-  private void populatePrimaryKeyarray(String[] dimInsertFileNames, Map<String, Boolean> map) {
-    List<String> primaryKeyList = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    for (String columnName : dimInsertFileNames) {
-      if (null != map.get(columnName)) {
-        map.put(columnName, false);
-      }
-    }
-    Set<Entry<String, Boolean>> entrySet = map.entrySet();
-    for (Entry<String, Boolean> entry : entrySet) {
-      if (entry.getValue()) {
-        primaryKeyList.add(entry.getKey().trim());
-      }
-    }
-    primaryKeyStringArray = primaryKeyList.toArray(new String[primaryKeyList.size()]);
-  }
-
-  /**
-   * update the
-   */
-  private void updatePrimaryKeyMaxSurrogateMap() {
-    Map<String, Boolean> primaryKeyMap = columnsInfo.getPrimaryKeyMap();
-    for (Entry<String, Boolean> entry : primaryKeyMap.entrySet()) {
-      if (!primaryKeyMap.get(entry.getKey())) {
-        int repeatedPrimaryFromLevels =
-            getRepeatedPrimaryFromLevels(dimInsertFileNames, entry.getKey());
-
-        if (null == primaryKeysMaxSurroagetMap) {
-          primaryKeysMaxSurroagetMap =
-              new HashMap<String, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        }
-        primaryKeysMaxSurroagetMap.put(entry.getKey(), max[repeatedPrimaryFromLevels]);
-      }
-    }
-  }
-
-  private int getRepeatedPrimaryFromLevels(String[] columnNames, String primaryKey) {
-    for (int j = 0; j < columnNames.length; j++) {
-      if (primaryKey.equals(columnNames[j])) {
-        return j;
-      }
-    }
-    return -1;
-  }
-
-  private String checkAndCreateLoadFolderNumber(String databaseName,
-      String tableName) throws IOException {
-    String carbonDataDirectoryPath = CarbonDataProcessorUtil
-        .getLocalDataFolderLocation(databaseName, tableName, taskNo, partitionID, segmentId + "",
-            false);
-    boolean isDirCreated = new File(carbonDataDirectoryPath).mkdirs();
-    if (!isDirCreated) {
-      throw new IOException("Unable to create data load directory" + carbonDataDirectoryPath);
-    }
-    return carbonDataDirectoryPath;
-  }
-
-  /**
-   * This method will update the maxkey information.
-   * @param tabColumnName
-   * @param maxKey max cardinality of a column
-   */
-  private void updateMaxKeyInfo(String tabColumnName, int maxKey) {
-    checkAndUpdateMap(maxKey, tabColumnName);
-  }
-
-  /**
-   * This method will generate cache for all the global dictionaries during data loading.
-   */
-  private void populateCache() throws IOException {
-    String carbonStorePath =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS);
-    String[] dimColumnNames = columnsInfo.getDimColNames();
-    String[] dimColumnIds = columnsInfo.getDimensionColumnIds();
-    String databaseName = columnsInfo.getDatabaseName();
-    String tableName = columnsInfo.getTableName();
-    CarbonTable carbonTable = CarbonMetadata.getInstance()
-        .getCarbonTable(databaseName + CarbonCommonConstants.UNDERSCORE + tableName);
-    CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    Cache reverseDictionaryCache =
-        cacheProvider.createCache(CacheType.REVERSE_DICTIONARY, carbonStorePath);
-    List<String> dictionaryKeys = new ArrayList<>(dimColumnNames.length);
-    List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
-        new ArrayList<>(dimColumnNames.length);
-    ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper =
-        columnsInfo.getColumnSchemaDetailsWrapper();
-    // update the member cache for dimension
-    for (int i = 0; i < dimColumnNames.length; i++) {
-      String dimColName = dimColumnNames[i].substring(tableName.length() + 1);
-      ColumnSchemaDetails details = columnSchemaDetailsWrapper.get(dimColumnIds[i]);
-      if (details.isDirectDictionary()) {
-        continue;
-      }
-      GenericDataType complexType = columnsInfo.getComplexTypesMap().get(dimColName);
-      if (complexType != null) {
-        List<GenericDataType> primitiveChild = new ArrayList<GenericDataType>();
-        complexType.getAllPrimitiveChildren(primitiveChild);
-        for (GenericDataType eachPrimitive : primitiveChild) {
-          details = columnSchemaDetailsWrapper.get(eachPrimitive.getColumnId());
-          if (details.isDirectDictionary()) {
-            continue;
-          }
-          ColumnIdentifier columnIdentifier = new ColumnIdentifier(eachPrimitive.getColumnId(),
-              columnsInfo.getColumnProperties(eachPrimitive.getName()), details.getColumnType());
-          String dimColumnName =
-              tableName + CarbonCommonConstants.UNDERSCORE + eachPrimitive.getName();
-          DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-              new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier);
-          dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
-          dictionaryKeys.add(dimColumnName);
-        }
-      } else {
-        ColumnIdentifier columnIdentifier =
-            new ColumnIdentifier(dimColumnIds[i], columnsInfo.getColumnProperties(dimColName),
-                details.getColumnType());
-        DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-            new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier);
-        dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
-        dictionaryKeys.add(dimColumnNames[i]);
-      }
-    }
-    initDictionaryCacheInfo(dictionaryKeys, dictionaryColumnUniqueIdentifiers,
-        reverseDictionaryCache);
-  }
-
-  /**
-   * This method will initial the needed information for a dictionary of one column.
-   *
-   * @param dictionaryKeys
-   * @param dictionaryColumnUniqueIdentifiers
-   * @param reverseDictionaryCache
-   * @throws KettleException
-   */
-  private void initDictionaryCacheInfo(List<String> dictionaryKeys,
-      List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers,
-      Cache reverseDictionaryCache) throws IOException {
-    long lruCacheStartTime = System.currentTimeMillis();
-    List reverseDictionaries = reverseDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers);
-    for (int i = 0; i < reverseDictionaries.size(); i++) {
-      Dictionary reverseDictionary = (Dictionary) reverseDictionaries.get(i);
-      getDictionaryCaches().put(dictionaryKeys.get(i), reverseDictionary);
-      updateMaxKeyInfo(dictionaryKeys.get(i), reverseDictionary.getDictionaryChunks().getSize());
-    }
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordLruCacheLoadTime(
-        (System.currentTimeMillis() - lruCacheStartTime) / 1000.0);
-  }
-
-  @Override protected int getSurrogateFromStore(String value, int index, Object[] properties)
-      throws KettleException {
-    max[index]++;
-    int key = max[index];
-    return key;
-  }
-
-  @Override
-  protected int updateSurrogateToStore(String tuple, String columnName, int index, int key,
-      Object[] properties) throws KettleException {
-    Map<String, Integer> cache = getTimeDimCache().get(columnName);
-    if (cache == null) {
-      return key;
-    }
-    return key;
-  }
-
-  private void checkAndUpdateMap(int maxKey, String dimInsertFileNames) {
-    String[] dimsFiles2 = getDimsFiles();
-    for (int i = 0; i < dimsFiles2.length; i++) {
-      if (dimInsertFileNames.equalsIgnoreCase(dimsFiles2[i])) {
-        if (max[i] < maxKey) {
-          max[i] = maxKey;
-          break;
-        }
-      }
-    }
-
-  }
-
-  @Override public boolean isCacheFilled(String[] columns) {
-    for (String column : columns) {
-      Dictionary dicCache = getDictionaryCaches().get(column);
-      if (null == dicCache) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public IFileManagerComposite getFileManager() {
-    return fileManager;
-  }
-
-  @Override protected byte[] getNormalizedHierFromStore(int[] val, String hier, int primaryKey,
-      HierarchyValueWriterForCSV hierWriter) throws KettleException {
-    byte[] bytes;
-    try {
-      bytes = columnsInfo.getKeyGenerators().get(hier).generateKey(val);
-      hierWriter.getByteArrayList().add(new ByteArrayHolder(bytes, primaryKey));
-    } catch (KeyGenException e) {
-      throw new KettleException(e);
-    }
-    return bytes;
-  }
-
-  @Override public int getSurrogateForMeasure(String tuple, String columnName)
-      throws KettleException {
-    Integer measureSurrogate = null;
-    Map<String, Dictionary> dictionaryCaches = getDictionaryCaches();
-    Dictionary dicCache = dictionaryCaches.get(columnName);
-    measureSurrogate = dicCache.getSurrogateKey(tuple);
-    return measureSurrogate;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/dbbased/messages/messages_en_US.properties
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/dbbased/messages/messages_en_US.properties b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/dbbased/messages/messages_en_US.properties
deleted file mode 100644
index d5b97fa..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/dbbased/messages/messages_en_US.properties
+++ /dev/null
@@ -1,61 +0,0 @@
-#
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-#
-CarbonDialog.Shell.Title=Carbon Surrogate Generator
-CarbonDialog.Group.Label=Carbon Settings
-CarbonDialog.dim.Label=Dimensions
-CarbonDialog.dim.Tooltip=Dimension index separated by comma(,)
-CarbonDialog.msr.Label=Measures
-CarbonDialog.msr.Tooltip=Measures index separated by comma(,)
-CarbonDialog.hier.Label=Hierarichies
-CarbonDialog.hier.Tooltip=Hierarichies seperated by &.
-
-CarbonDialog.time.Label=Time Hierarichy
-CarbonDialog.time.Tooltip=Time Hierarichy seperated by ,
-
-CarbonDialog.loc.Label=Location
-CarbonDialog.loc.Tooltip=Location to keep hierarichy files
-
-CarbonDialog.con.Label=Connection
-CarbonDialog.con.Tooltip=Jdbc url
-
-CarbonDialog.Schema.Label=Schema
-
-CarbonDialog.BatchSize.Label=Carbon Batch Size
-CarbonDialog.BatchSize.Tooltip=Carbon Batch Size
-
-CarbonDialog.CarbonProperties.Label=Carbon Properties
-CarbonDialog.CarbonProperties.Tooltip=Carbon Properties
-
-CarbonDialog.MetaHier.Label=Carbon Metadata Hierarichies
-CarbonDialog.MetaHier.Tooltip=Carbon Metadata Hierarichies
-
-CarbonDialog.IsInitialLoad=Initial Load
-
-CarbonDialog.MetadataFolderLocation.Label= Carbon Metadata Files Location
-CarbonDialog.MetadataFolderLocation.Tooltip= Carbon Metadata Files Location
-
-CarbonDialog.DoMapping.UnableToFindInput=Cannot find previous fields
-CarbonStep.Exception.UnexpectedErrorInReadingStepInfo=Error reading step info
-CarbonStep.Exception.UnableToSaveStepInfoToRepository=Error saving step info
-
-CarbonStep.Error.UnableFindField=Unable to find key field
-CarbonStep.Check.StepIsReceivingInfoFromOtherSteps=Step is receiving input from other steps
-CarbonStep.Check.NoInputReceivedFromOtherSteps=Step is not receiving any input from other steps
-CarbonStep.Check.MissingFieldsNotFoundInInput=Required fields not found in input
-CarbonStep.Check.AllFieldsFoundInInput=All required fields found
-CarbonStep.Check.CouldNotReadFromPreviousSteps=Could not read from previous steps

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 bde8215..27662a4 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
@@ -55,12 +55,6 @@ import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
 import org.apache.carbondata.processing.newflow.DataField;
 
 import org.apache.commons.lang3.ArrayUtils;
-import org.pentaho.di.core.CheckResult;
-import org.pentaho.di.core.CheckResultInterface;
-import org.pentaho.di.core.Const;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.i18n.BaseMessages;
-import org.pentaho.di.trans.step.StepMeta;
 
 public final class CarbonDataProcessorUtil {
   private static final LogService LOGGER =
@@ -149,9 +143,7 @@ public final class CarbonDataProcessorUtil {
 
     String badRecordsInProgressFileName = null;
     String changedFileName = null;
-    // CHECKSTYLE:OFF
     for (CarbonFile badFiles : listFiles) {
-      // CHECKSTYLE:ON
       badRecordsInProgressFileName = badFiles.getName();
 
       changedFileName = badLogStoreLocation + File.separator + badRecordsInProgressFileName
@@ -164,51 +156,6 @@ public final class CarbonDataProcessorUtil {
           LOGGER.error("Unable to delete File : " + badFiles.getName());
         }
       }
-    } // CHECKSTYLE:ON
-  }
-
-  public static void checkResult(List<CheckResultInterface> remarks, StepMeta stepMeta,
-      String[] input) {
-    CheckResult cr;
-
-    // See if we have input streams leading to this step!
-    if (input.length > 0) {
-      cr = new CheckResult(CheckResult.TYPE_RESULT_OK, "Step is receiving info from other steps.",
-          stepMeta);
-      remarks.add(cr);
-    } else {
-      cr = new CheckResult(CheckResult.TYPE_RESULT_ERROR, "No input received from other steps!",
-          stepMeta);
-      remarks.add(cr);
-    }
-  }
-
-  public static void check(Class<?> pkg, List<CheckResultInterface> remarks, StepMeta stepMeta,
-      RowMetaInterface prev, String[] input) {
-    CheckResult cr;
-
-    // See if we have input streams leading to this step!
-    if (input.length > 0) {
-      cr = new CheckResult(CheckResult.TYPE_RESULT_OK,
-          BaseMessages.getString(pkg, "CarbonStep.Check.StepIsReceivingInfoFromOtherSteps"),
-          stepMeta);
-      remarks.add(cr);
-    } else {
-      cr = new CheckResult(CheckResult.TYPE_RESULT_ERROR,
-          BaseMessages.getString(pkg, "CarbonStep.Check.NoInputReceivedFromOtherSteps"), stepMeta);
-      remarks.add(cr);
-    }
-
-    // also check that each expected key fields are acually coming
-    if (prev != null && prev.size() > 0) {
-      cr = new CheckResult(CheckResultInterface.TYPE_RESULT_OK,
-          BaseMessages.getString(pkg, "CarbonStep.Check.AllFieldsFoundInInput"), stepMeta);
-      remarks.add(cr);
-    } else {
-      String errorMessage =
-          BaseMessages.getString(pkg, "CarbonStep.Check.CouldNotReadFromPreviousSteps") + Const.CR;
-      cr = new CheckResult(CheckResultInterface.TYPE_RESULT_ERROR, errorMessage, stepMeta);
-      remarks.add(cr);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/util/CarbonSchemaParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonSchemaParser.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonSchemaParser.java
deleted file mode 100644
index bba40e9..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonSchemaParser.java
+++ /dev/null
@@ -1,1076 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedHashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.keygenerator.factory.KeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.datatype.DataType;
-import org.apache.carbondata.core.metadata.encoder.Encoding;
-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.util.CarbonUtil;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema.DimensionRelation;
-import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetails;
-import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper;
-
-public final class CarbonSchemaParser {
-  /**
-   *
-   */
-  public static final String QUOTES = "\"";
-
-  private CarbonSchemaParser() {
-
-  }
-
-  /**
-   * This method Return the dimension queries based on quotest required or not.
-   *
-   * @param dimensions
-   * @return
-   */
-  public static String getDimensionSQLQueries(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema, boolean isQuotesRequired, String quote) {
-    if (isQuotesRequired) {
-      return getDimensionSQLQueriesWithQuotes(dimensions, carbonDataLoadSchema, quote);
-    } else {
-      return getDimensionSQLQueries(dimensions, carbonDataLoadSchema);
-    }
-  }
-
-  public static String getDenormColNames(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    //
-    List<String> foreignKeys = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    Set<String> allRelationCols = new HashSet<String>();
-
-    for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) {
-      foreignKeys.add(dimensionRelation.getRelation().getFactForeignKeyColumn());
-      allRelationCols.addAll(dimensionRelation.getColumns());
-    }
-
-    StringBuilder columns = new StringBuilder();
-
-    for (CarbonDimension dim : dimensions) {
-      if (foreignKeys.contains(dim.getColName()) && !allRelationCols.contains(dim.getColName())) {
-        columns.append(dim.getColName());
-        columns.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      }
-    }
-
-    String columnstr = columns.toString();
-    if (columnstr.length() > 0 && columnstr.endsWith(CarbonCommonConstants.HASH_SPC_CHARACTER)) {
-      columnstr = columnstr
-          .substring(0, columnstr.length() - CarbonCommonConstants.HASH_SPC_CHARACTER.length());
-    }
-
-    return columnstr;
-  }
-
-  private static String getDimensionSQLQueries(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    //
-    List<String> queryList = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    for (CarbonDimension dim : dimensions) {
-
-      String tableName = extractDimensionTableName(dim.getColName(), carbonDataLoadSchema);
-      StringBuilder query;
-      String factTableName = carbonDataLoadSchema.getCarbonTable().getFactTableName();
-      if (factTableName.equals(tableName)) {
-        continue;
-      }
-      String dimName = dim.getColName();
-      query =
-          new StringBuilder(dimName + '_' + tableName + CarbonCommonConstants.COLON_SPC_CHARACTER);
-
-      String primaryKey = null;
-      for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) {
-        for (String field : dimensionRelation.getColumns()) {
-          if (dimName.equals(field)) {
-            primaryKey = dimensionRelation.getRelation().getDimensionPrimaryKeyColumn();
-            break;
-          }
-        }
-        if (null != primaryKey) {
-          break;
-        }
-      }
-      query.append("SELECT ");
-      query.append(primaryKey + ',');
-      query.append(dimName);
-      query.append(" FROM " + tableName);
-      queryList.add(query.toString());
-    }
-    StringBuilder finalQuryString = new StringBuilder();
-
-    for (int i = 0; i < queryList.size() - 1; i++) {
-      finalQuryString.append(queryList.get(i));
-      finalQuryString.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-    }
-    if (queryList.size() > 0) {
-      finalQuryString.append(queryList.get(queryList.size() - 1));
-    }
-    return finalQuryString.toString();
-  }
-
-  /**
-   * @param table
-   * @param dimensions
-   * @return
-   */
-
-  private static String getDimensionSQLQueriesWithQuotes(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema, String quotes) {
-    //
-    List<String> queryList = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    //        Property[] properties = null;
-    for (CarbonDimension dim : dimensions) {
-
-      String tableName = extractDimensionTableName(dim.getColName(), carbonDataLoadSchema);
-      StringBuilder query;
-      String factTableName = carbonDataLoadSchema.getCarbonTable().getFactTableName();
-      if (factTableName.equals(tableName)) {
-        continue;
-      }
-      String dimName = dim.getColName();
-      query =
-          new StringBuilder(dimName + '_' + tableName + CarbonCommonConstants.COLON_SPC_CHARACTER);
-
-      String primaryKey = null;
-      for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) {
-        for (String field : dimensionRelation.getColumns()) {
-          if (dimName.equals(field)) {
-            primaryKey = dimensionRelation.getRelation().getDimensionPrimaryKeyColumn();
-            break;
-          }
-        }
-        if (null != primaryKey) {
-          break;
-        }
-      }
-      query.append("SELECT ");
-      query.append(quotes + primaryKey + quotes + ',');
-      query.append(quotes + dimName + quotes);
-      query.append(" FROM " + quotes + tableName + quotes);
-      queryList.add(query.toString());
-    }
-    StringBuilder finalQuryString = new StringBuilder();
-
-    for (int i = 0; i < queryList.size() - 1; i++) {
-      finalQuryString.append(queryList.get(i));
-      finalQuryString.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-    }
-    if (queryList.size() > 0) {
-      finalQuryString.append(queryList.get(queryList.size() - 1));
-    }
-    return finalQuryString.toString();
-  }
-
-  /**
-   * @param dimensions
-   * @param measures
-   * @param factTableName
-   * @param isQuotesRequired
-   * @param schemaInfo
-   * @return
-   */
-  public static String getTableInputSQLQuery(List<CarbonDimension> dimensions,
-      List<CarbonMeasure> measures, String factTableName, boolean isQuotesRequired,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    StringBuilder query = new StringBuilder("SELECT ");
-
-    getQueryForDimension(dimensions, query, factTableName, isQuotesRequired, carbonDataLoadSchema);
-
-    if (!"select".equalsIgnoreCase(query.toString().trim())) {
-      query.append(",");
-    }
-    Set<String> uniqueMsrCols = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (int i = 0; i < measures.size(); i++) {
-
-      uniqueMsrCols.add(measures.get(i).getColName());
-    }
-    String[] uniqueMeasure = uniqueMsrCols.toArray(new String[uniqueMsrCols.size()]);
-    for (int j = 0; j < uniqueMeasure.length; j++) {
-      query.append(System.getProperty("line.separator"));
-      if (isQuotesRequired) {
-        query.append(QUOTES + uniqueMeasure[j] + QUOTES);
-      } else {
-        query.append(uniqueMeasure[j]);
-      }
-
-      if (j != uniqueMeasure.length - 1) {
-
-        query.append(",");
-      }
-    }
-    query.append(System.getProperty("line.separator"));
-
-    if (isQuotesRequired) {
-      query.append(" FROM " + QUOTES + factTableName + QUOTES + ' ');
-    } else {
-      query.append(" FROM " + factTableName + ' ');
-    }
-
-    return query.toString();
-  }
-
-  private static void getQueryForDimension(List<CarbonDimension> dimensions, StringBuilder query,
-      String factTableName, boolean isQuotesRequired, CarbonDataLoadSchema carbonDataLoadSchema) {
-    int counter = 0;
-    for (CarbonDimension cDim : dimensions) {
-
-      String foreignKey = null;
-      for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) {
-        for (String field : dimensionRelation.getColumns()) {
-          if (cDim.getColName().equals(field)) {
-            foreignKey = dimensionRelation.getRelation().getFactForeignKeyColumn();
-          }
-        }
-      }
-      if (foreignKey != null) {
-        query.append(System.getProperty("line.separator"));
-        if (counter != 0) {
-          query.append(',');
-        }
-
-        if (isQuotesRequired) {
-          query.append(QUOTES + foreignKey + QUOTES);
-        } else {
-          query.append(foreignKey);
-        }
-        continue;
-      } else {
-        query.append(System.getProperty("line.separator"));
-        if (counter != 0) {
-          query.append(',');
-        }
-
-        if (isQuotesRequired) {
-          query.append(QUOTES + factTableName + QUOTES + '.' + QUOTES + cDim.getColName() + QUOTES);
-        } else {
-          query.append(factTableName + '.' + cDim.getColName());
-        }
-      }
-      counter++;
-    }
-  }
-
-  /**
-   * Get dimension string from a array of TableDimension,which can be shared
-   * TableDimension within schema or in a table.
-   *
-   * @param table
-   * @param dimensions
-   * @return
-   */
-  public static int getDimensionString(List<CarbonDimension> dimensions, StringBuilder dimString,
-      int counter, CarbonDataLoadSchema carbonDataLoadSchema) {
-    for (CarbonDimension cDimension : dimensions) {
-      if (!cDimension.getEncoder().contains(Encoding.DICTIONARY)) {
-        continue;
-      }
-
-      String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      dimString.append(
-          tableName + '_' + cDimension.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER
-              + counter + CarbonCommonConstants.COLON_SPC_CHARACTER + -1
-              + CarbonCommonConstants.COLON_SPC_CHARACTER + 'Y'
-              + CarbonCommonConstants.COMA_SPC_CHARACTER);
-      counter++;
-    }
-    return counter;
-  }
-
-  /**
-   * Return mapping of Column name to cardinality
-   */
-
-  public static Map<String, String> getCardinalities(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    Map<String, String> cardinalities = new LinkedHashMap<String, String>();
-    for (CarbonDimension cDimension : dimensions) {
-      String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      cardinalities.put(tableName + '_' + cDimension.getColName(), -1 + "");
-    }
-    return cardinalities;
-  }
-
-  /**
-   * Get measure string from a array of Measure
-   *
-   * @param measures
-   * @return
-   */
-  public static String getMeasureString(List<CarbonMeasure> measures, int counter) {
-    StringBuilder measureString = new StringBuilder();
-    int i = measures.size();
-    for (CarbonMeasure measure : measures) {
-
-      measureString
-          .append(measure.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER + counter);
-      counter++;
-      if (i > 1) {
-        measureString.append(CarbonCommonConstants.COMA_SPC_CHARACTER);
-      }
-      i--;
-
-    }
-    return measureString.toString();
-  }
-
-  /**
-   * Get measure string from a array of Measure
-   *
-   * @param measures
-   * @return
-   */
-  public static String[] getMeasures(List<CarbonMeasure> measures) {
-    String[] measuresStringArray = new String[measures.size()];
-
-    for (int i = 0; i < measuresStringArray.length; i++) {
-      measuresStringArray[i] = measures.get(i).getColName();
-    }
-    return measuresStringArray;
-  }
-
-  //TODO SIMIAN
-
-  /**
-   * Get hierarchy string from dimensions
-   *
-   * @param dimensions
-   * @return
-   */
-  public static String getHierarchyString(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    StringBuilder hierString = new StringBuilder();
-    String hierStr = "";
-
-    for (CarbonDimension cDimension : dimensions) {
-      if (cDimension.getEncoder().contains(Encoding.DICTIONARY)) {
-        continue;
-      }
-      String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      String cDimName = cDimension.getColName();
-      hierStr = 0 + CarbonCommonConstants.AMPERSAND_SPC_CHARACTER;
-      hierStr = cDimName + '_' + tableName + CarbonCommonConstants.COLON_SPC_CHARACTER + hierStr;
-      hierString.append(hierStr);
-    }
-
-    hierStr = hierString.toString();
-    if (hierStr.length() > 0 && hierStr.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      hierStr = hierStr
-          .substring(0, hierStr.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-    return hierStr;
-  }
-
-  /**
-   * this method will return table columns
-   *
-   * @param dimensions
-   * @param carbonDataLoadSchema
-   * @return
-   */
-  public static String[] getTableDimensions(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    List<String> list = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    for (CarbonDimension cDimension : dimensions) {
-      // Ignoring the dimensions which are high cardinality dimension
-      if (!cDimension.getEncoder().contains(Encoding.DICTIONARY)) {
-        continue;
-      }
-      list.add(extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema) + "_"
-          + cDimension.getColName());
-    }
-    String[] fields = new String[list.size()];
-    fields = list.toArray(fields);
-    return fields;
-  }
-
-  /**
-   * This method will extract dimension table name,
-   * By default, fact table name will be returned.
-   *
-   * @param dimensionColName
-   * @param carbonDataLoadSchema
-   * @return
-   */
-  private static String extractDimensionTableName(String dimensionColName,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    List<DimensionRelation> dimensionRelationList = carbonDataLoadSchema.getDimensionRelationList();
-
-    for (DimensionRelation dimensionRelation : dimensionRelationList) {
-      for (String field : dimensionRelation.getColumns()) {
-        if (dimensionColName.equals(field)) {
-          return dimensionRelation.getTableName();
-        }
-      }
-    }
-    return carbonDataLoadSchema.getCarbonTable().getFactTableName();
-  }
-
-  /**
-   * It will return all column groups in below format
-   * 0,1~2~3,4,5,6~7~8,9
-   * groups are
-   * ,-> all ordinal with different group id
-   * ~-> all ordinal with same group id
-   *
-   * @param dimensions
-   * @return
-   */
-  public static String getColumnGroups(List<CarbonDimension> dimensions) {
-    StringBuilder columnGroups = new StringBuilder();
-    for (int i = 0; i < dimensions.size(); i++) {
-      CarbonDimension dimension = dimensions.get(i);
-      //assuming complex dimensions will always be atlast
-      if (null != dimension.getListOfChildDimensions() &&
-          dimension.getListOfChildDimensions().size() > 0) {
-        break;
-      }
-      if (!dimension.hasEncoding(Encoding.DICTIONARY)) {
-        continue;
-      }
-      columnGroups.append(dimension.getOrdinal());
-      if (i < dimensions.size() - 1) {
-        int currGroupOrdinal = dimension.columnGroupId();
-        int nextGroupOrdinal = dimensions.get(i + 1).columnGroupId();
-        if (currGroupOrdinal == nextGroupOrdinal && currGroupOrdinal != -1) {
-          columnGroups.append("~");
-        } else {
-          columnGroups.append(",");
-        }
-      }
-
-    }
-    return columnGroups.toString();
-  }
-
-  /**
-   * getHeirAndCardinalityString
-   *
-   * @param dimensions
-   * @param schema
-   * @return String
-   */
-  public static String getHeirAndCardinalityString(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    StringBuilder builder = new StringBuilder();
-    String heirName = null;
-    for (CarbonDimension cDimension : dimensions) {
-      heirName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      String dimName = cDimension.getColName();
-      builder.append(dimName + '_' + heirName + ".hierarchy");
-      builder.append(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      builder.append(-1);
-      builder.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-    return builder.toString();
-  }
-
-  /**
-   * @param dimensions
-   * @return
-   */
-  public static String getMetaHeirString(List<CarbonDimension> dimensions, CarbonTable schema) {
-    StringBuilder propString = new StringBuilder();
-    String tableName = schema.getFactTableName();
-    for (CarbonDimension cDimension : dimensions) {
-      propString.append(tableName + "_" + cDimension.getColName());
-      propString.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-
-    }
-    // Delete the last special character
-    String prop = propString.toString();
-    if (prop.endsWith(CarbonCommonConstants.HASH_SPC_CHARACTER)) {
-      prop = prop.substring(0, prop.length() - CarbonCommonConstants.HASH_SPC_CHARACTER.length());
-    }
-    return prop;
-  }
-
-  public static String getTableNameString(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    StringBuilder stringBuffer = new StringBuilder();
-
-    for (CarbonDimension cDimension : dimensions) {
-      String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-
-      stringBuffer.append(cDimension.getColName() + '_' + cDimension.getColName());
-      stringBuffer.append(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      stringBuffer.append(tableName);
-      stringBuffer.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-    // Delete the last & character
-    String string = stringBuffer.toString();
-    if (string.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      string = string
-          .substring(0, string.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-    return string;
-  }
-
-  /**
-   * This method will concatenate all the column ids for a given list of dimensions
-   *
-   * @param dimensions
-   * @return
-   */
-  public static String getColumnIdString(List<CarbonDimension> dimensions) {
-    StringBuilder stringBuffer = new StringBuilder();
-    for (CarbonDimension cDimension : dimensions) {
-      if (!cDimension.hasEncoding(Encoding.DICTIONARY)) {
-        continue;
-      }
-      stringBuffer.append(cDimension.getColumnId());
-      stringBuffer.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-    // Delete the last & character
-    String string = stringBuffer.toString();
-    if (string.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      string = string
-          .substring(0, string.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-    return string;
-  }
-
-  /**
-   * @param dimensions
-   * @param schema
-   * @return
-   */
-  public static String getMdkeySizeForFact(List<CarbonDimension> dimensions) {
-    int[] dims = new int[dimensions.size()];
-    for (int i = 0; i < dims.length; i++) {
-      dims[i] = -1;
-    }
-    return KeyGeneratorFactory.getKeyGenerator(dims).getKeySizeInBytes() + "";
-  }
-
-  /**
-   * @param dimensions
-   * @param schema
-   * @return
-   */
-  public static String getHeirAndKeySizeMapForFact(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    StringBuilder stringBuffer = new StringBuilder();
-    String heirName = null;
-    int[] dims = null;
-    int keySizeInBytes = 0;
-    for (CarbonDimension cDimension : dimensions) {
-      String dimName = cDimension.getColName();
-      heirName = extractDimensionTableName(dimName, carbonDataLoadSchema);
-      dims = new int[] { -1 };
-      keySizeInBytes = KeyGeneratorFactory.getKeyGenerator(dims).getKeySizeInBytes();
-      stringBuffer.append(dimName + '_' + heirName + CarbonCommonConstants.HIERARCHY_FILE_EXTENSION
-          + CarbonCommonConstants.COLON_SPC_CHARACTER + keySizeInBytes
-          + CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-    return stringBuffer.toString();
-  }
-
-  /**
-   * @param dimensions
-   * @return
-   */
-  public static String getHierarchyStringWithColumnNames(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-
-    StringBuilder hierString = new StringBuilder();
-    String hierStr = "";
-
-    for (CarbonDimension cDimension : dimensions) {
-      if (cDimension.getEncoder().contains(Encoding.DICTIONARY)) {
-        continue;
-      }
-      String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      String cDimName = cDimension.getColName();
-      hierStr = cDimName + CarbonCommonConstants.AMPERSAND_SPC_CHARACTER;
-      hierStr = cDimName + '_' + tableName + CarbonCommonConstants.COLON_SPC_CHARACTER + hierStr;
-      hierString.append(hierStr);
-    }
-
-    hierStr = hierString.toString();
-    if (hierStr.length() > 0 && hierStr.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      hierStr = hierStr
-          .substring(0, hierStr.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-    return hierStr;
-
-  }
-
-  /**
-   * Return foreign key array
-   *
-   * @param dimensions
-   * @return
-   */
-  public static String[] getForeignKeyForTables(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    Set<String> foreignKey = new LinkedHashSet<String>();
-    for (CarbonDimension cDimension : dimensions) {
-
-      List<DimensionRelation> dimensionRelationList =
-          carbonDataLoadSchema.getDimensionRelationList();
-
-      for (DimensionRelation dimensionRelation : dimensionRelationList) {
-        for (String field : dimensionRelation.getColumns()) {
-          if (cDimension.getColName().equals(field)) {
-            foreignKey.add(dimensionRelation.getRelation().getFactForeignKeyColumn());
-          }
-        }
-      }
-
-    }
-    return foreignKey.toArray(new String[foreignKey.size()]);
-
-  }
-
-  /**
-   * Return foreign key and respective hierarchy String.
-   *
-   * @param dimensions
-   * @return
-   */
-  public static String getForeignKeyHierarchyString(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema, String factTable) {
-    StringBuilder foreignKeyHierarchyString = new StringBuilder();
-    String columns = "";
-
-    for (CarbonDimension cDimension : dimensions) {
-      String dimTableName =
-          extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      String dimName = cDimension.getColName();
-
-      if (dimTableName.equals(factTable)) {
-        continue;
-      }
-
-      String foreignKey = null;
-      for (DimensionRelation dimensionRelation : carbonDataLoadSchema.getDimensionRelationList()) {
-        for (String field : dimensionRelation.getColumns()) {
-          if (dimName.equals(field)) {
-            foreignKey = dimensionRelation.getRelation().getFactForeignKeyColumn();
-            break;
-          }
-        }
-
-        foreignKeyHierarchyString.append(foreignKey);
-        foreignKeyHierarchyString.append(CarbonCommonConstants.COLON_SPC_CHARACTER);
-        foreignKeyHierarchyString.append(dimName + '_' + dimTableName);
-        foreignKeyHierarchyString.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-      }
-    }
-    columns = foreignKeyHierarchyString.toString();
-    if (columns.length() > 0 && columns.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      columns = columns
-          .substring(0, columns.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-    return columns;
-
-  }
-
-  /**
-   * Return foreign key and respective hierarchy String.
-   *
-   * @param dimensions
-   * @param factTableName
-   * @return
-   */
-  public static String getForeignKeyAndPrimaryKeyMapString(
-      List<DimensionRelation> dimensionRelationList) {
-    StringBuilder foreignKeyHierarchyString = new StringBuilder();
-    String columns = "";
-
-    for (DimensionRelation dimensionRelation : dimensionRelationList) {
-      foreignKeyHierarchyString.append(dimensionRelation.getRelation().getFactForeignKeyColumn());
-      foreignKeyHierarchyString.append(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      foreignKeyHierarchyString.append(
-          dimensionRelation.getTableName() + '_' + dimensionRelation.getRelation()
-              .getDimensionPrimaryKeyColumn());
-      foreignKeyHierarchyString.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-    columns = foreignKeyHierarchyString.toString();
-    if (columns.length() > 0 && columns.endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      columns = columns
-          .substring(0, columns.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-    return columns;
-
-  }
-
-  /**
-   * Return foreign key array
-   *
-   * @param dimensions
-   * @return
-   */
-  public static String getPrimaryKeyString(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    StringBuilder primaryKeyStringbuffer = new StringBuilder();
-    for (CarbonDimension cDimension : dimensions) {
-      String dimTableName =
-          extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      String dimName = cDimension.getColName();
-
-      String primaryKey = null;
-      if (dimTableName.equals(carbonDataLoadSchema.getCarbonTable().getFactTableName())) {
-        dimTableName = dimName;
-      } else {
-        for (DimensionRelation dimensionRelation : carbonDataLoadSchema
-            .getDimensionRelationList()) {
-          for (String field : dimensionRelation.getColumns()) {
-            if (field.equals(dimName)) {
-              primaryKey = dimensionRelation.getRelation().getDimensionPrimaryKeyColumn();
-              break;
-            }
-          }
-        }
-      }
-
-      primaryKeyStringbuffer.append(dimTableName + '_' + primaryKey);
-      primaryKeyStringbuffer.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-
-    }
-
-    String primaryKeyString = primaryKeyStringbuffer.toString();
-
-    if (primaryKeyString.length() > 0 && primaryKeyString
-        .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      primaryKeyString = primaryKeyString.substring(0,
-          primaryKeyString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-
-    return primaryKeyString;
-  }
-
-  /**
-   * Get Measure Name String
-   *
-   * @param table
-   * @return
-   */
-  public static String getMeasuresNamesString(List<CarbonMeasure> measures) {
-    StringBuilder measureNames = new StringBuilder();
-
-    for (int i = 0; i < measures.size(); i++) {
-      measureNames.append(measures.get(i).getColName());
-      measureNames.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-
-    String measureNameString = measureNames.toString();
-
-    if (measureNameString.length() > 0 && measureNameString
-        .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      measureNameString = measureNameString.substring(0,
-          measureNameString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-
-    return measureNameString;
-  }
-
-  /**
-   * Get Measure Name String
-   *
-   * @param table
-   * @return
-   */
-  public static String getMeasuresUniqueColumnNamesString(List<CarbonMeasure> measures) {
-    StringBuilder measureNames = new StringBuilder();
-    Set<String> set = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (int i = 0; i < measures.size(); i++) {
-      if (!set.contains(measures.get(i).getColName())) {
-        set.add(measures.get(i).getColName());
-        measureNames.append(measures.get(i).getColName());
-        measureNames.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-      }
-    }
-    String measureNameString = measureNames.toString();
-    if (measureNameString.length() > 0 && measureNameString
-        .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      measureNameString = measureNameString.substring(0,
-          measureNameString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-    return measureNameString;
-  }
-
-  /**
-   * Get Measure Aggregator array
-   *
-   * @param table
-   * @return
-   */
-  public static String[] getMeasuresAggragatorArray(List<CarbonMeasure> measures) {
-    String[] msrAggregators = new String[measures.size()];
-
-    for (int i = 0; i < msrAggregators.length; i++) {
-      msrAggregators[i] = "sum";
-    }
-
-    return msrAggregators;
-  }
-
-  /**
-   * @param schemaInfo
-   * @param table
-   * @return
-   */
-  public static String getActualDimensions(List<CarbonDimension> dimensions) {
-    StringBuilder actualDim = new StringBuilder();
-    for (CarbonDimension cDimension : dimensions) {
-      if (!cDimension.getEncoder().contains(Encoding.DICTIONARY)) {
-        continue;
-      }
-      actualDim.append(cDimension.getColName());
-      actualDim.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-
-    String actualDimString = actualDim.toString();
-
-    if (actualDimString.length() > 0 && actualDimString
-        .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      actualDimString = actualDimString.substring(0,
-          actualDimString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-
-    return actualDimString;
-  }
-
-  /**Method will prepare column name and its data type string inorder
-   * to pass to the ETL steps.
-   * @param schemaInfo
-   * @param table
-   * @return
-   */
-  public static String getDimensionsDataTypes(List<CarbonDimension> dimensions) {
-    StringBuilder dimDataTypeBuilder = new StringBuilder();
-    for (CarbonDimension cDimension : dimensions) {
-      dimDataTypeBuilder.append(cDimension.getColName());
-      dimDataTypeBuilder.append(CarbonCommonConstants.COMA_SPC_CHARACTER);
-      dimDataTypeBuilder.append(cDimension.getDataType().toString());
-      dimDataTypeBuilder.append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-    String dimDataType = dimDataTypeBuilder.toString();
-    return dimDataType;
-  }
-
-  public static String getMeasuresDataType(List<CarbonMeasure> measures) {
-    StringBuilder measureDataTypeString = new StringBuilder();
-
-    for (CarbonMeasure measure : measures) {
-      measureDataTypeString.append(measure.getDataType())
-          .append(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    }
-
-    String measureTypeString = measureDataTypeString.toString();
-
-    if (measureTypeString.length() > 0 && measureTypeString
-        .endsWith(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER)) {
-      measureTypeString = measureTypeString.substring(0,
-          measureTypeString.length() - CarbonCommonConstants.AMPERSAND_SPC_CHARACTER.length());
-    }
-
-    return measureTypeString;
-
-  }
-
-  /**
-   * Below method will be used to get the level and its data type string
-   *
-   * @param dimensions
-   * @param schema
-   * @param table
-   * @return String
-   */
-  public static String getLevelAndDataTypeMapString(List<CarbonDimension> dimensions,
-      CarbonDataLoadSchema carbonDataLoadSchema) {
-    StringBuilder dimString = new StringBuilder();
-    for (CarbonDimension cDimension : dimensions) {
-      String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      String levelName = tableName + '_' + cDimension.getColName();
-      dimString.append(levelName + CarbonCommonConstants.LEVEL_FILE_EXTENSION
-          + CarbonCommonConstants.COLON_SPC_CHARACTER + cDimension.getDataType()
-          + CarbonCommonConstants.HASH_SPC_CHARACTER);
-    }
-    return dimString.toString();
-  }
-
-  /**
-   * Below method will be used to get the complex dimension string
-   *
-   * @param dimensions
-   * @param schema
-   * @param table
-   * @return String
-   */
-  public static String getComplexTypeString(List<CarbonDimension> dimensions) {
-    StringBuilder dimString = new StringBuilder();
-    for (int i = 0; i < dimensions.size(); i++) {
-      CarbonDimension dimension = dimensions.get(i);
-      if (dimension.getDataType().equals(DataType.ARRAY) || dimension.getDataType()
-          .equals(DataType.STRUCT)) {
-        addAllComplexTypeChildren(dimension, dimString, "");
-        dimString.append(CarbonCommonConstants.SEMICOLON_SPC_CHARACTER);
-      }
-    }
-    return dimString.toString();
-  }
-
-  /**
-   * This method will return all the child dimensions under complex dimension
-   *
-   * @param dimension
-   * @param dimString
-   * @param parent
-   */
-  private static void addAllComplexTypeChildren(CarbonDimension dimension, StringBuilder dimString,
-      String parent) {
-    dimString.append(
-        dimension.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER + dimension.getDataType()
-            + CarbonCommonConstants.COLON_SPC_CHARACTER + parent
-            + CarbonCommonConstants.COLON_SPC_CHARACTER + dimension.getColumnId()
-            + CarbonCommonConstants.HASH_SPC_CHARACTER);
-    for (int i = 0; i < dimension.getNumberOfChild(); i++) {
-      CarbonDimension childDim = dimension.getListOfChildDimensions().get(i);
-      if (childDim.getNumberOfChild() > 0) {
-        addAllComplexTypeChildren(childDim, dimString, dimension.getColName());
-      } else {
-        dimString.append(
-            childDim.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER + childDim
-                .getDataType() + CarbonCommonConstants.COLON_SPC_CHARACTER + dimension.getColName()
-                + CarbonCommonConstants.COLON_SPC_CHARACTER + childDim.getColumnId()
-                + CarbonCommonConstants.COLON_SPC_CHARACTER + childDim.getOrdinal()
-                + CarbonCommonConstants.HASH_SPC_CHARACTER);
-      }
-    }
-  }
-
-  /**
-   * the method returns the ColumnSchemaDetailsWrapper
-   *
-   * @param dimensions
-   * @return
-   */
-  public static ColumnSchemaDetailsWrapper getColumnSchemaDetails(
-      List<CarbonDimension> dimensions) {
-    ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper = new ColumnSchemaDetailsWrapper();
-    Map<String, ColumnSchemaDetails> columnSchemaDetailsMap =
-        new HashMap<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    fillColumnSchemaDetailsWithComplex(dimensions, columnSchemaDetailsMap);
-    columnSchemaDetailsWrapper.setColumnSchemaDetailsMap(columnSchemaDetailsMap);
-    return columnSchemaDetailsWrapper;
-  }
-
-  private static void fillColumnSchemaDetailsWithComplex(
-      List<CarbonDimension> dimensions,
-      Map<String, ColumnSchemaDetails> columnSchemaDetailsMap) {
-    for (CarbonDimension cDimension : dimensions) {
-      ColumnSchemaDetails details =
-          new ColumnSchemaDetails(cDimension.getColName(), cDimension.getDataType(),
-          CarbonUtil.hasEncoding(cDimension.getEncoder(), Encoding.DIRECT_DICTIONARY));
-      columnSchemaDetailsMap.put(cDimension.getColumnSchema().getColumnUniqueId(), details);
-      if (cDimension.isComplex()) {
-        fillColumnSchemaDetailsWithComplex(cDimension.getListOfChildDimensions(),
-            columnSchemaDetailsMap);
-      }
-    }
-  }
-
-  /**
-   * Get dimension string from a array of TableDimension,which can be shared
-   * TableDimension within schema or in a table.
-   *
-   * @param table
-   * @param dimensions
-   * @return
-   */
-  public static int getNoDictionaryDimensionString(List<CarbonDimension> dimensions,
-      StringBuilder dimString, int counter, CarbonDataLoadSchema carbonDataLoadSchema) {
-    for (CarbonDimension cDimension : dimensions) {
-      if (cDimension.getEncoder().contains(Encoding.DICTIONARY)) {
-        continue;
-      }
-
-      String tableName = extractDimensionTableName(cDimension.getColName(), carbonDataLoadSchema);
-      dimString.append(
-          tableName + '_' + cDimension.getColName() + CarbonCommonConstants.COLON_SPC_CHARACTER
-              + counter + CarbonCommonConstants.COLON_SPC_CHARACTER + -1
-              + CarbonCommonConstants.COLON_SPC_CHARACTER + 'Y'
-              + CarbonCommonConstants.COMA_SPC_CHARACTER);
-      counter++;
-    }
-    return counter;
-  }
-
-  public static String getColumnPropertiesString(List<CarbonDimension> dimensions) {
-    StringBuilder colPropertiesString = new StringBuilder();
-    for (int dim = 0; dim < dimensions.size(); dim++) {
-      CarbonDimension dimension = dimensions.get(dim);
-      if (dimension.isComplex()) {
-        List<CarbonDimension> childs = dimension.getListOfChildDimensions();
-        for (CarbonDimension child : childs) {
-          buildDimensionColumnPropertyString(child, colPropertiesString, dim);
-        }
-      } else {
-        buildDimensionColumnPropertyString(dimension, colPropertiesString, dim);
-      }
-
-    }
-
-    return colPropertiesString.toString();
-  }
-
-  protected static void buildDimensionColumnPropertyString(CarbonDimension dimension,
-      StringBuilder colPropertiesString, int dim) {
-    Map<String, String> columnProperties = dimension.getColumnProperties();
-    if (null != columnProperties && columnProperties.size() > 0) {
-      if (colPropertiesString.length() > 0) {
-        colPropertiesString.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
-      }
-      colPropertiesString.append(dimension.getColName())
-          .append(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      int size = columnProperties.entrySet().size();
-      int index = 0;
-      Iterator<Entry<String, String>> itr = columnProperties.entrySet().iterator();
-      while (itr.hasNext()) {
-        Entry<String, String> entry = itr.next();
-        colPropertiesString.append(entry.getKey())
-            .append(CarbonCommonConstants.HYPHEN_SPC_CHARACTER);
-        colPropertiesString.append(entry.getValue());
-        index++;
-        if (index < size) {
-          colPropertiesString.append(CarbonCommonConstants.COMA_SPC_CHARACTER);
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/HadoopFileInputMeta.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/HadoopFileInputMeta.java b/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/HadoopFileInputMeta.java
deleted file mode 100644
index 15b0ac8..0000000
--- a/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/HadoopFileInputMeta.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.pentaho.di.trans.steps.hadoopfileinput;
-
-import org.pentaho.di.core.annotations.Step;
-import org.pentaho.di.trans.steps.textfileinput.TextFileInputMeta;
-
-@Step(id = "HadoopFileInputPlugin", image = "HDI.png", name = "HadoopFileInputPlugin",
-    description = "Process files from an HDFS location", categoryDescription = "Hadoop")
-public class HadoopFileInputMeta extends TextFileInputMeta {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/messages/messages_en_US.properties
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/messages/messages_en_US.properties b/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/messages/messages_en_US.properties
deleted file mode 100644
index 4ed60dd..0000000
--- a/processing/src/main/java/org/pentaho/di/trans/steps/hadoopfileinput/messages/messages_en_US.properties
+++ /dev/null
@@ -1,18 +0,0 @@
-#
-#  Licensed to the Apache Software Foundation (ASF) under one
-#  or more contributor license agreements.  See the NOTICE file
-#  distributed with this work for additional information
-#  regarding copyright ownership.  The ASF licenses this file
-#  to you under the Apache License, Version 2.0 (the
-#  "License"); you may not use this file except in compliance
-#  with the License.  You may obtain a copy of the License at
-#
-#      http://www.apache.org/licenses/LICENSE-2.0
-#
-#  Unless required by applicable law or agreed to in writing, software
-#  distributed under the License is distributed on an "AS IS" BASIS,
-#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-#  See the License for the specific language governing permissions and
-# limitations under the License.
-#
-HadoopFileInputDialog.DialogTitle=Hadoop File Input
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
index 714a32f..156c57e 100644
--- a/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
+++ b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
@@ -40,7 +40,7 @@ import org.apache.carbondata.core.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.datastore.block.TableBlockUniqueIdentifier;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.test.util.StoreCreator;
+import org.apache.carbondata.processing.StoreCreator;
 
 import junit.framework.TestCase;
 import org.junit.AfterClass;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java b/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
index 4bb7d16..68d822b 100644
--- a/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
+++ b/processing/src/test/java/org/apache/carbondata/lcm/locks/LocalFileLockTest.java
@@ -20,9 +20,9 @@ import org.apache.carbondata.core.metadata.CarbonTableIdentifier;
 import org.apache.carbondata.core.locks.LocalFileLock;
 import org.apache.carbondata.core.locks.LockUsage;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.pentaho.di.core.util.Assert;
 
 /**
  * Test class to test the functionality of the local file locking.


[05/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepMeta.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepMeta.java b/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepMeta.java
deleted file mode 100644
index 2b54812..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/sortandgroupby/sortdatastep/SortKeyStepMeta.java
+++ /dev/null
@@ -1,490 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.sortdatastep;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import org.pentaho.di.core.CheckResultInterface;
-import org.pentaho.di.core.Counter;
-import org.pentaho.di.core.database.DatabaseMeta;
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.exception.KettleXMLException;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.core.xml.XMLHandler;
-import org.pentaho.di.i18n.BaseMessages;
-import org.pentaho.di.repository.ObjectId;
-import org.pentaho.di.repository.Repository;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStepMeta;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-import org.w3c.dom.Node;
-
-public class SortKeyStepMeta extends BaseStepMeta implements StepMetaInterface {
-  /**
-   * PKG
-   */
-  private static final Class<?> PKG = SortKeyStepMeta.class;
-
-  /**
-   * tabelName
-   */
-  private String tabelName;
-
-  /**
-   * outputRowSize
-   */
-  private String outputRowSize;
-
-  /**
-   * tableName
-   */
-  private String tableName;
-
-  /**
-   * databaseName
-   */
-  private String databaseName;
-
-  /**
-   * Dimension Count
-   */
-  private String dimensionCount;
-
-  /**
-   * ComplexTypes Count
-   */
-  private String complexDimensionCount;
-
-  /**
-   * Dimension Count
-   */
-  private int noDictionaryCount;
-
-  /**
-   * measureCount
-   */
-  private String measureCount;
-
-  private String factDimLensString;
-
-  /**
-   * isUpdateMemberRequest
-   */
-  private String updateMemberRequest;
-
-  private String measureDataType;
-
-  private String noDictionaryDims;
-  /**
-   * partitionID
-   */
-  private String partitionID;
-  /**
-   * Id of the load folder
-   */
-  private String segmentId;
-  /**
-   * task id, each spark task has a unique id
-   */
-  private String taskNo;
-  /**
-   * To determine the column whether is dictionary or not.
-   */
-  private String noDictionaryDimsMapping;
-
-  /**
-   * set the default value for all the properties
-   */
-  @Override public void setDefault() {
-    this.tabelName = "";
-    factDimLensString = "";
-    outputRowSize = "";
-    databaseName = "";
-    noDictionaryDims = "";
-    noDictionaryDimsMapping = "";
-    tableName = "";
-    dimensionCount = "";
-    complexDimensionCount = "";
-    measureCount = "";
-    updateMemberRequest = "";
-    measureDataType = "";
-    partitionID = "";
-    segmentId = "";
-    taskNo = "";
-  }
-
-  /**
-   * Get the XML that represents the values in this step
-   *
-   * @return the XML that represents the metadata in this step
-   * @throws KettleException in case there is a conversion or XML encoding error
-   */
-  public String getXML() {
-    StringBuilder retval = new StringBuilder(150);
-    retval.append("    ").append(XMLHandler.addTagValue("TableName", this.tabelName));
-    retval.append("    ").append(XMLHandler.addTagValue("factDimLensString", factDimLensString));
-    retval.append("    ").append(XMLHandler.addTagValue("outputRowSize", this.outputRowSize));
-    retval.append("    ").append(XMLHandler.addTagValue("tableName", this.tableName));
-    retval.append("    ").append(XMLHandler.addTagValue("databaseName", this.databaseName));
-    retval.append("    ").append(XMLHandler.addTagValue("dimensionCount", this.dimensionCount));
-    retval.append("    ").append(XMLHandler.addTagValue("noDictionaryDims", this.noDictionaryDims));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("noDictionaryDimsMapping", this.noDictionaryDimsMapping));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("complexDimensionCount", this.complexDimensionCount));
-    retval.append("    ").append(XMLHandler.addTagValue("measureCount", this.measureCount));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue("isUpdateMemberRequest", this.updateMemberRequest));
-    retval.append("    ").append(XMLHandler.addTagValue("measureDataType", measureDataType));
-    retval.append("    ").append(XMLHandler.addTagValue("partitionID", partitionID));
-    retval.append("    ").append(XMLHandler.addTagValue("segmentId", segmentId));
-    retval.append("    ").append(XMLHandler.addTagValue("taskNo", taskNo));
-    return retval.toString();
-  }
-
-  /**
-   * Load the values for this step from an XML Node
-   *
-   * @param stepnode  the Node to get the info from
-   * @param databases The available list of databases to reference to
-   * @param counters  Counters to reference.
-   * @throws KettleXMLException When an unexpected XML error occurred. (malformed etc.)
-   */
-  public void loadXML(Node stepnode, List<DatabaseMeta> databases, Map<String, Counter> counters)
-      throws KettleXMLException {
-    try {
-      this.tabelName = XMLHandler.getTagValue(stepnode, "TableName");
-      this.outputRowSize = XMLHandler.getTagValue(stepnode, "outputRowSize");
-      this.factDimLensString = XMLHandler.getTagValue(stepnode, "factDimLensString");
-      this.tableName = XMLHandler.getTagValue(stepnode, "tableName");
-      this.databaseName = XMLHandler.getTagValue(stepnode, "databaseName");
-      this.dimensionCount = XMLHandler.getTagValue(stepnode, "dimensionCount");
-      this.noDictionaryDims = XMLHandler.getTagValue(stepnode, "noDictionaryDims");
-      this.noDictionaryDimsMapping = XMLHandler.getTagValue(stepnode, "noDictionaryDimsMapping");
-      this.complexDimensionCount = XMLHandler.getTagValue(stepnode, "complexDimensionCount");
-      this.measureCount = XMLHandler.getTagValue(stepnode, "measureCount");
-      this.updateMemberRequest = XMLHandler.getTagValue(stepnode, "isUpdateMemberRequest");
-      this.measureDataType = XMLHandler.getTagValue(stepnode, "measureDataType");
-      this.partitionID = XMLHandler.getTagValue(stepnode, "partitionID");
-      this.segmentId = XMLHandler.getTagValue(stepnode, "segmentId");
-      this.taskNo = XMLHandler.getTagValue(stepnode, "taskNo");
-    } catch (Exception e) {
-      throw new KettleXMLException("Unable to read step info from XML node", e);
-    }
-  }
-
-  /**
-   * Save the steps data into a Kettle repository
-   *
-   * @param rep              The Kettle repository to save to
-   * @param idTransformation The transformation ID
-   * @param idStep           The step ID
-   * @throws KettleException When an unexpected error occurred (database, network, etc)
-   */
-  public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep)
-      throws KettleException {
-    try {
-      rep.saveStepAttribute(idTransformation, idStep, "TableName", this.tabelName);
-
-      rep.saveStepAttribute(idTransformation, idStep, "factDimLensString", factDimLensString);
-      rep.saveStepAttribute(idTransformation, idStep, "outputRowSize", this.outputRowSize);
-      rep.saveStepAttribute(idTransformation, idStep, "tableName", this.tableName);
-      rep.saveStepAttribute(idTransformation, idStep, "databaseName", this.databaseName);
-      rep.saveStepAttribute(idTransformation, idStep, "dimensionCount", this.dimensionCount);
-      rep.saveStepAttribute(idTransformation, idStep, "noDictionaryDims", this.noDictionaryDims);
-      rep.saveStepAttribute(idTransformation, idStep, "noDictionaryDimsMapping",
-          this.noDictionaryDimsMapping);
-      rep.saveStepAttribute(idTransformation, idStep, "complexDimensionCount",
-          this.complexDimensionCount);
-      rep.saveStepAttribute(idTransformation, idStep, "measureCount", this.measureCount);
-      rep.saveStepAttribute(idTransformation, idStep, "isUpdateMemberRequest",
-          this.updateMemberRequest);
-      rep.saveStepAttribute(idTransformation, idStep, "measureDataType", measureDataType);
-      rep.saveStepAttribute(idTransformation, idStep, "partitionID", partitionID);
-      rep.saveStepAttribute(idTransformation, idStep, "segmentId", segmentId);
-      rep.saveStepAttribute(idTransformation, idStep, "taskNo", taskNo);
-    } catch (Exception e) {
-      throw new KettleException(BaseMessages
-          .getString(PKG, "TemplateStep.Exception.UnableToSaveStepInfoToRepository", new String[0])
-          + idStep, e);
-    }
-  }
-
-  /**
-   * Read the steps information from a Kettle repository
-   *
-   * @param rep       The repository to read from
-   * @param idStep    The step ID
-   * @param databases The databases to reference
-   * @param counters  The counters to reference
-   * @throws KettleException When an unexpected error occurred (database, network, etc)
-   */
-  public void readRep(Repository rep, ObjectId idStep, List<DatabaseMeta> databases,
-      Map<String, Counter> counters) throws KettleException {
-    try {
-      this.tabelName = rep.getStepAttributeString(idStep, "TableName");
-      this.outputRowSize = rep.getStepAttributeString(idStep, "outputRowSize");
-      this.databaseName = rep.getStepAttributeString(idStep, "databaseName");
-      this.tableName = rep.getStepAttributeString(idStep, "tableName");
-      this.dimensionCount = rep.getStepAttributeString(idStep, "dimensionCount");
-      this.noDictionaryDims = rep.getStepAttributeString(idStep, "noDictionaryDims");
-      this.noDictionaryDims = rep.getStepAttributeString(idStep, "noDictionaryDimsMapping");
-      this.complexDimensionCount = rep.getStepAttributeString(idStep, "complexDimensionCount");
-      this.measureCount = rep.getStepAttributeString(idStep, "measureCount");
-      this.updateMemberRequest = rep.getStepAttributeString(idStep, "isUpdateMemberRequest");
-      this.measureDataType = rep.getStepAttributeString(idStep, "measureDataType");
-      this.partitionID = rep.getStepAttributeString(idStep, "partitionID");
-      this.segmentId = rep.getStepAttributeString(idStep, "segmentId");
-      this.taskNo = rep.getStepAttributeString(idStep, "taskNo");
-    } catch (Exception ex) {
-      throw new KettleException(BaseMessages
-          .getString(PKG, "CarbonDataWriterStepMeta.Exception.UnexpectedErrorInReadingStepInfo",
-              new String[0]), ex);
-    }
-  }
-
-  /**
-   * Checks the settings of this step and puts the findings in a remarks List.
-   *
-   * @param remarks  The list to put the remarks in @see
-   *                 org.pentaho.di.core.CheckResult
-   * @param stepMeta The stepMeta to help checking
-   * @param prev     The fields coming from the previous step
-   * @param input    The input step names
-   * @param output   The output step names
-   * @param info     The fields that are used as information by the step
-   */
-  public void check(List<CheckResultInterface> remarks, TransMeta transMeta, StepMeta stepMeta,
-      RowMetaInterface prev, String[] input, String[] output, RowMetaInterface info) {
-    CarbonDataProcessorUtil.checkResult(remarks, stepMeta, input);
-  }
-
-  /**
-   * Get the executing step, needed by Trans to launch a step.
-   *
-   * @param stepMeta          The step info
-   * @param stepDataInterface the step data interface linked to this step. Here the step can
-   *                          store temporary data, database connections, etc.
-   * @param copyNr            The copy nr to get
-   * @param transMeta         The transformation info
-   * @param trans             The launching transformation
-   */
-  public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr,
-      TransMeta transMeta, Trans trans) {
-    return new SortKeyStep(stepMeta, stepDataInterface, copyNr, transMeta, trans);
-  }
-
-  /**
-   * Get a new instance of the appropriate data class. This data class
-   * implements the StepDataInterface. It basically contains the persisting
-   * data that needs to live on, even if a worker thread is terminated.
-   *
-   * @return The appropriate StepDataInterface class.
-   */
-  public StepDataInterface getStepData() {
-    return new SortKeyStepData();
-  }
-
-  /**
-   * Below method will be used to get the out row size
-   *
-   * @return outputRowSize
-   */
-  public String getOutputRowSize() {
-    return outputRowSize;
-  }
-
-  /**
-   * below mthod will be used to set the out row size
-   *
-   * @param outputRowSize
-   */
-  public void setOutputRowSize(String outputRowSize) {
-    this.outputRowSize = outputRowSize;
-  }
-
-  /**
-   * This method will return the table name
-   *
-   * @return tabelName
-   */
-
-  public String getTabelName() {
-    return this.tabelName;
-  }
-
-  /**
-   * This method will set the table name
-   *
-   * @param tabelName
-   */
-  public void setTabelName(String tabelName) {
-    this.tabelName = tabelName;
-  }
-
-  /**
-   * @param tableName the tableName to set
-   */
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
-
-  /**
-   * @return the databaseName
-   */
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  /**
-   * @param databaseName the databaseName to set
-   */
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * @return the dimensionCount
-   */
-  public int getDimensionCount() {
-    return Integer.parseInt(dimensionCount);
-  }
-
-  public void setDimensionCount(String dimensionCount) {
-    this.dimensionCount = dimensionCount;
-  }
-
-  /**
-   * @return the complexDimensionCount
-   */
-  public int getComplexDimensionCount() {
-    return Integer.parseInt(complexDimensionCount);
-  }
-
-  public void setComplexDimensionCount(String complexDimensionCount) {
-    this.complexDimensionCount = complexDimensionCount;
-  }
-
-  /**
-   * @return the measureCount
-   */
-  public int getMeasureCount() {
-    return Integer.parseInt(measureCount);
-  }
-
-  /**
-   * @param measureCount the measureCount to set
-   */
-  public void setMeasureCount(String measureCount) {
-    this.measureCount = measureCount;
-  }
-
-  /**
-   * @param isUpdateMemberRequest the isUpdateMemberRequest to set
-   */
-  public void setIsUpdateMemberRequest(String isUpdateMemberRequest) {
-    this.updateMemberRequest = isUpdateMemberRequest;
-  }
-
-  public void setMeasureDataType(String measureDataType) {
-    this.measureDataType = measureDataType;
-  }
-
-  public String getNoDictionaryDims() {
-    return noDictionaryDims;
-  }
-
-  public void setNoDictionaryDims(String noDictionaryDims) {
-    this.noDictionaryDims = noDictionaryDims;
-  }
-
-  /**
-   * @return the noDictionaryCount
-   */
-  public int getNoDictionaryCount() {
-    return noDictionaryCount;
-  }
-
-  /**
-   * @param noDictionaryCount the noDictionaryCount to set
-   */
-  public void setNoDictionaryCount(int noDictionaryCount) {
-    this.noDictionaryCount = noDictionaryCount;
-  }
-
-  /**
-   * @return partitionId
-   */
-  public String getPartitionID() {
-    return partitionID;
-  }
-
-  /**
-   * @param partitionID
-   */
-  public void setPartitionID(String partitionID) {
-    this.partitionID = partitionID;
-  }
-
-  /**
-   * return segmentId
-   *
-   * @return
-   */
-  public String getSegmentId() {
-    return segmentId;
-  }
-
-  /**
-   * set segment Id
-   *
-   * @param segmentId
-   */
-  public void setSegmentId(String segmentId) {
-    this.segmentId = segmentId;
-  }
-
-  /**
-   * @param taskNo
-   */
-  public void setTaskNo(String taskNo) {
-    this.taskNo = taskNo;
-  }
-
-  /**
-   * @return
-   */
-  public String getTaskNo() {
-    return taskNo;
-  }
-
-  public String getNoDictionaryDimsMapping() {
-    return noDictionaryDimsMapping;
-  }
-
-  public void setNoDictionaryDimsMapping(String noDictionaryDimsMapping) {
-    this.noDictionaryDimsMapping = noDictionaryDimsMapping;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 3806c55..2affa03 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
@@ -65,13 +65,13 @@ import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.NodeHolder;
 import org.apache.carbondata.core.util.ValueCompressionUtil;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.mdkeygen.file.FileManager;
-import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.apache.carbondata.processing.store.colgroup.ColGroupBlockStorage;
 import org.apache.carbondata.processing.store.colgroup.ColGroupDataHolder;
 import org.apache.carbondata.processing.store.colgroup.ColGroupMinMax;
 import org.apache.carbondata.processing.store.colgroup.ColumnDataHolder;
 import org.apache.carbondata.processing.store.colgroup.DataHolder;
+import org.apache.carbondata.processing.store.file.FileManager;
+import org.apache.carbondata.processing.store.file.IFileManagerComposite;
 import org.apache.carbondata.processing.store.writer.CarbonDataWriterVo;
 import org.apache.carbondata.processing.store.writer.CarbonFactDataWriter;
 import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException;
@@ -256,8 +256,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
    */
   private boolean compactionFlow;
 
-  private boolean useKettle;
-
   private int bucketNumber;
 
   private long schemaUpdatedTimeStamp;
@@ -279,7 +277,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     this.dimensionCount = carbonFactDataHandlerModel.getDimensionCount();
     this.complexIndexMap = carbonFactDataHandlerModel.getComplexIndexMap();
     this.primitiveDimLens = carbonFactDataHandlerModel.getPrimitiveDimLens();
-    this.useKettle = carbonFactDataHandlerModel.isUseKettle();
     this.isAggKeyBlock = Boolean.parseBoolean(CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.AGGREAGATE_COLUMNAR_KEY_BLOCK,
             CarbonCommonConstants.AGGREAGATE_COLUMNAR_KEY_BLOCK_DEFAULTVALUE));
@@ -481,144 +478,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     }
   }
 
-  // TODO remove after kettle flow is removed
-  private NodeHolder processDataRows(List<Object[]> dataRows) throws CarbonDataWriterException {
-    Object[] max = new Object[measureCount];
-    Object[] min = new Object[measureCount];
-    int[] decimal = new int[measureCount];
-    Object[] uniqueValue = new Object[measureCount];
-    // to store index of the measure columns which are null
-    BitSet[] nullValueIndexBitSet = getMeasureNullValueIndexBitSet(measureCount);
-    for (int i = 0; i < max.length; i++) {
-      if (type[i] == CarbonCommonConstants.BIG_INT_MEASURE) {
-        max[i] = Long.MIN_VALUE;
-      } else if (type[i] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) {
-        max[i] = -Double.MAX_VALUE;
-      } else if (type[i] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) {
-        max[i] = new BigDecimal(-Double.MAX_VALUE);
-      } else {
-        max[i] = 0.0;
-      }
-    }
-    for (int i = 0; i < min.length; i++) {
-      if (type[i] == CarbonCommonConstants.BIG_INT_MEASURE) {
-        min[i] = Long.MAX_VALUE;
-        uniqueValue[i] = Long.MIN_VALUE;
-      } else if (type[i] == CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE) {
-        min[i] = Double.MAX_VALUE;
-        uniqueValue[i] = Double.MIN_VALUE;
-      } else if (type[i] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) {
-        min[i] = new BigDecimal(Double.MAX_VALUE);
-        uniqueValue[i] = new BigDecimal(Double.MIN_VALUE);
-      } else {
-        min[i] = 0.0;
-        uniqueValue[i] = 0.0;
-      }
-    }
-    for (int i = 0; i < decimal.length; i++) {
-      decimal[i] = 0;
-    }
-
-    byte[] startKey = null;
-    byte[] endKey = null;
-    byte[] noDictStartKey = null;
-    byte[] noDictEndKey = null;
-    CarbonWriteDataHolder[] dataHolder = initialiseDataHolder(dataRows.size());
-    CarbonWriteDataHolder keyDataHolder = initialiseKeyBlockHolder(dataRows.size());
-    CarbonWriteDataHolder noDictionaryKeyDataHolder = null;
-    if ((noDictionaryCount + complexColCount) > 0) {
-      noDictionaryKeyDataHolder = initialiseKeyBlockHolder(dataRows.size());
-    }
-
-    for (int count = 0; count < dataRows.size(); count++) {
-      Object[] row = dataRows.get(count);
-      byte[] mdKey = (byte[]) row[this.mdKeyIndex];
-      byte[] noDictionaryKey = null;
-      if (noDictionaryCount > 0 || complexIndexMap.size() > 0) {
-        noDictionaryKey = (byte[]) row[this.mdKeyIndex - 1];
-      }
-      ByteBuffer byteBuffer = null;
-      byte[] b = null;
-      if (count == 0) {
-        startKey = mdKey;
-        noDictStartKey = noDictionaryKey;
-      }
-      endKey = mdKey;
-      noDictEndKey = noDictionaryKey;
-      // add to key store
-      if (mdKey.length > 0) {
-        keyDataHolder.setWritableByteArrayValueByIndex(count, mdKey);
-      }
-      // for storing the byte [] for high card.
-      if (noDictionaryCount > 0 || complexIndexMap.size() > 0) {
-        noDictionaryKeyDataHolder.setWritableByteArrayValueByIndex(count, noDictionaryKey);
-      }
-      //Add all columns to keyDataHolder
-      keyDataHolder.setWritableByteArrayValueByIndex(count, this.mdKeyIndex, row);
-      // CHECKSTYLE:OFF Approval No:Approval-351
-      for (int k = 0; k < otherMeasureIndex.length; k++) {
-        if (type[otherMeasureIndex[k]] == CarbonCommonConstants.BIG_INT_MEASURE) {
-          if (null == row[otherMeasureIndex[k]]) {
-            nullValueIndexBitSet[otherMeasureIndex[k]].set(count);
-            dataHolder[otherMeasureIndex[k]].setWritableLongValueByIndex(count, 0L);
-          } else {
-            dataHolder[otherMeasureIndex[k]]
-                .setWritableLongValueByIndex(count, row[otherMeasureIndex[k]]);
-          }
-        } else {
-          if (null == row[otherMeasureIndex[k]]) {
-            nullValueIndexBitSet[otherMeasureIndex[k]].set(count);
-            dataHolder[otherMeasureIndex[k]].setWritableDoubleValueByIndex(count, 0.0);
-          } else {
-            dataHolder[otherMeasureIndex[k]]
-                .setWritableDoubleValueByIndex(count, row[otherMeasureIndex[k]]);
-          }
-        }
-      }
-      calculateMaxMin(max, min, decimal, otherMeasureIndex, row);
-      for (int i = 0; i < customMeasureIndex.length; i++) {
-        if (null == row[customMeasureIndex[i]]
-            && type[customMeasureIndex[i]] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) {
-          BigDecimal val = BigDecimal.valueOf(0);
-          b = DataTypeUtil.bigDecimalToByte(val);
-          nullValueIndexBitSet[customMeasureIndex[i]].set(count);
-        } else {
-          if (this.compactionFlow) {
-            BigDecimal bigDecimal = ((Decimal) row[customMeasureIndex[i]]).toJavaBigDecimal();
-            b = DataTypeUtil.bigDecimalToByte(bigDecimal);
-          } else {
-            b = (byte[]) row[customMeasureIndex[i]];
-          }
-        }
-        byteBuffer = ByteBuffer.allocate(b.length + CarbonCommonConstants.INT_SIZE_IN_BYTE);
-        byteBuffer.putInt(b.length);
-        byteBuffer.put(b);
-        byteBuffer.flip();
-        b = byteBuffer.array();
-        dataHolder[customMeasureIndex[i]].setWritableByteArrayValueByIndex(count, b);
-      }
-      calculateMaxMin(max, min, decimal, customMeasureIndex, row);
-    }
-    calculateUniqueValue(min, uniqueValue);
-    byte[][] byteArrayValues = keyDataHolder.getByteArrayValues().clone();
-    byte[][] noDictionaryValueHolder = null;
-    if ((noDictionaryCount + complexColCount) > 0) {
-      noDictionaryValueHolder = noDictionaryKeyDataHolder.getByteArrayValues();
-    }
-    WriterCompressModel compressionModel = ValueCompressionUtil
-        .getWriterCompressModel(max, min, decimal, uniqueValue, type, new byte[max.length]);
-    byte[][] writableMeasureDataArray =
-        StoreFactory.createDataStore(compressionModel).getWritableMeasureDataArray(dataHolder)
-            .clone();
-    NodeHolder nodeHolder =
-        getNodeHolderObject(writableMeasureDataArray, byteArrayValues, dataRows.size(), startKey,
-            endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey,
-            nullValueIndexBitSet);
-    LOGGER.info("Number Of records processed: " + dataRows.size());
-    return nodeHolder;
-  }
-
-  private NodeHolder processDataRowsWithOutKettle(List<Object[]> dataRows)
+  private NodeHolder processDataRows(List<Object[]> dataRows)
       throws CarbonDataWriterException {
     Object[] max = new Object[measureCount];
     Object[] min = new Object[measureCount];
@@ -661,7 +521,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     byte[][] noDictStartKey = null;
     byte[][] noDictEndKey = null;
     CarbonWriteDataHolder[] dataHolder = initialiseDataHolder(dataRows.size());
-    CarbonWriteDataHolder keyDataHolder = initialiseKeyBlockHolderWithOutKettle(dataRows.size());
+    CarbonWriteDataHolder keyDataHolder = initialiseKeyBlockHolder(dataRows.size());
     CarbonWriteDataHolder noDictionaryKeyDataHolder = null;
     if ((noDictionaryCount + complexColCount) > 0) {
       noDictionaryKeyDataHolder = initialiseKeyBlockHolderForNonDictionary(dataRows.size());
@@ -756,142 +616,14 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
         StoreFactory.createDataStore(compressionModel).getWritableMeasureDataArray(dataHolder)
             .clone();
     NodeHolder nodeHolder =
-        getNodeHolderObjectWithOutKettle(writableMeasureDataArray, byteArrayValues, dataRows.size(),
+        getNodeHolderObject(writableMeasureDataArray, byteArrayValues, dataRows.size(),
             startKey, endKey, compressionModel, noDictionaryValueHolder, noDictStartKey,
             noDictEndKey, nullValueIndexBitSet);
     LOGGER.info("Number Of records processed: " + dataRows.size());
     return nodeHolder;
   }
 
-  // TODO remove after kettle flow is removed
-  private NodeHolder getNodeHolderObject(byte[][] dataHolderLocal, byte[][] byteArrayValues,
-      int entryCountLocal, byte[] startkeyLocal, byte[] endKeyLocal,
-      WriterCompressModel compressionModel, byte[][] noDictionaryData, byte[] noDictionaryStartKey,
-      byte[] noDictionaryEndKey, BitSet[] nullValueIndexBitSet) throws CarbonDataWriterException {
-    byte[][][] noDictionaryColumnsData = null;
-    List<ArrayList<byte[]>> colsAndValues = new ArrayList<ArrayList<byte[]>>();
-    int complexColCount = getComplexColsCount();
-
-    for (int i = 0; i < complexColCount; i++) {
-      colsAndValues.add(new ArrayList<byte[]>());
-    }
-    int noOfColumn = colGrpModel.getNoOfColumnStore();
-    DataHolder[] dataHolders = getDataHolders(noOfColumn, byteArrayValues.length);
-    for (int i = 0; i < byteArrayValues.length; i++) {
-      byte[][] splitKey = columnarSplitter.splitKey(byteArrayValues[i]);
-
-      for (int j = 0; j < splitKey.length; j++) {
-        dataHolders[j].addData(splitKey[j], i);
-      }
-    }
-    if (noDictionaryCount > 0 || complexIndexMap.size() > 0) {
-      noDictionaryColumnsData = new byte[noDictionaryCount][noDictionaryData.length][];
-      for (int i = 0; i < noDictionaryData.length; i++) {
-        int complexColumnIndex = primitiveDimLens.length + noDictionaryCount;
-        byte[][] splitKey = NonDictionaryUtil
-            .splitNoDictionaryKey(noDictionaryData[i], noDictionaryCount + complexIndexMap.size());
-
-        int complexTypeIndex = 0;
-        for (int j = 0; j < splitKey.length; j++) {
-          //nodictionary Columns
-          if (j < noDictionaryCount) {
-            noDictionaryColumnsData[j][i] = splitKey[j];
-          }
-          //complex types
-          else {
-            // Need to write columnar block from complex byte array
-            int index = complexColumnIndex - noDictionaryCount;
-            GenericDataType complexDataType = complexIndexMap.get(index);
-            complexColumnIndex++;
-            if (complexDataType != null) {
-              List<ArrayList<byte[]>> columnsArray = new ArrayList<ArrayList<byte[]>>();
-              for (int k = 0; k < complexDataType.getColsCount(); k++) {
-                columnsArray.add(new ArrayList<byte[]>());
-              }
-
-              try {
-                ByteBuffer complexDataWithoutBitPacking = ByteBuffer.wrap(splitKey[j]);
-                byte[] complexTypeData = new byte[complexDataWithoutBitPacking.getShort()];
-                complexDataWithoutBitPacking.get(complexTypeData);
-
-                ByteBuffer byteArrayInput = ByteBuffer.wrap(complexTypeData);
-                ByteArrayOutputStream byteArrayOutput = new ByteArrayOutputStream();
-                DataOutputStream dataOutputStream = new DataOutputStream(byteArrayOutput);
-                complexDataType
-                    .parseAndBitPack(byteArrayInput, dataOutputStream, this.complexKeyGenerator);
-                complexDataType.getColumnarDataForComplexType(columnsArray,
-                    ByteBuffer.wrap(byteArrayOutput.toByteArray()));
-                byteArrayOutput.close();
-              } catch (IOException e) {
-                throw new CarbonDataWriterException(
-                    "Problem while bit packing and writing complex datatype", e);
-              } catch (KeyGenException e) {
-                throw new CarbonDataWriterException(
-                    "Problem while bit packing and writing complex datatype", e);
-              }
-
-              for (ArrayList<byte[]> eachColumn : columnsArray) {
-                colsAndValues.get(complexTypeIndex++).addAll(eachColumn);
-              }
-            } else {
-              // This case not possible as ComplexType is the last columns
-            }
-          }
-        }
-      }
-    }
-    thread_pool_size = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.NUM_CORES_BLOCK_SORT,
-            CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL));
-    ExecutorService executorService = Executors.newFixedThreadPool(thread_pool_size);
-    List<Future<IndexStorage>> submit = new ArrayList<Future<IndexStorage>>(
-        primitiveDimLens.length + noDictionaryCount + complexColCount);
-    int i = 0;
-    int dictionaryColumnCount = -1;
-    int noDictionaryColumnCount = -1;
-    for (i = 0; i < dimensionType.length; i++) {
-      if (dimensionType[i]) {
-        dictionaryColumnCount++;
-        if (colGrpModel.isColumnar(dictionaryColumnCount)) {
-          submit.add(executorService.submit(
-              new BlockSortThread(i, dataHolders[dictionaryColumnCount].getData(), true,
-                  isUseInvertedIndex[i])));
-        } else {
-          submit.add(
-              executorService.submit(new ColGroupBlockStorage(dataHolders[dictionaryColumnCount])));
-        }
-      } else {
-        submit.add(executorService.submit(
-            new BlockSortThread(i, noDictionaryColumnsData[++noDictionaryColumnCount], false, true,
-                true, isUseInvertedIndex[i])));
-      }
-    }
-    for (int k = 0; k < complexColCount; k++) {
-      submit.add(executorService.submit(new BlockSortThread(i++,
-          colsAndValues.get(k).toArray(new byte[colsAndValues.get(k).size()][]), false, true)));
-    }
-    executorService.shutdown();
-    try {
-      executorService.awaitTermination(1, TimeUnit.DAYS);
-    } catch (InterruptedException e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    IndexStorage[] blockStorage =
-        new IndexStorage[colGrpModel.getNoOfColumnStore() + noDictionaryCount + complexColCount];
-    try {
-      for (int k = 0; k < blockStorage.length; k++) {
-        blockStorage[k] = submit.get(k).get();
-      }
-    } catch (Exception e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    return this.dataWriter
-        .buildDataNodeHolder(blockStorage, dataHolderLocal, entryCountLocal, startkeyLocal,
-            endKeyLocal, compressionModel, noDictionaryStartKey, noDictionaryEndKey,
-            nullValueIndexBitSet);
-  }
-
-  private NodeHolder getNodeHolderObjectWithOutKettle(byte[][] dataHolderLocal,
+  private NodeHolder getNodeHolderObject(byte[][] dataHolderLocal,
       byte[][] byteArrayValues, int entryCountLocal, byte[] startkeyLocal, byte[] endKeyLocal,
       WriterCompressModel compressionModel, byte[][][] noDictionaryData,
       byte[][] noDictionaryStartKey, byte[][] noDictionaryEndKey, BitSet[] nullValueIndexBitSet)
@@ -1343,16 +1075,9 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     return blockKeySizeWithComplexTypes;
   }
 
-  // TODO Remove after kettle flow got removed.
   private CarbonWriteDataHolder initialiseKeyBlockHolder(int size) {
     CarbonWriteDataHolder keyDataHolder = new CarbonWriteDataHolder();
-    keyDataHolder.initialiseByteArrayValues(size);
-    return keyDataHolder;
-  }
-
-  private CarbonWriteDataHolder initialiseKeyBlockHolderWithOutKettle(int size) {
-    CarbonWriteDataHolder keyDataHolder = new CarbonWriteDataHolder();
-    keyDataHolder.initialiseByteArrayValuesWithOutKettle(size);
+    keyDataHolder.initialiseByteArrayValuesForKey(size);
     return keyDataHolder;
   }
 
@@ -1556,12 +1281,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
      */
     @Override public Void call() throws Exception {
       try {
-        NodeHolder nodeHolder;
-        if (useKettle) {
-          nodeHolder = processDataRows(dataRows);
-        } else {
-          nodeHolder = processDataRowsWithOutKettle(dataRows);
-        }
+        NodeHolder nodeHolder = processDataRows(dataRows);
         // insert the object in array according to sequence number
         int indexInNodeHolderArray = (sequenceNumber - 1) % numberOfCores;
         blockletDataHolder.put(nodeHolder, indexInNodeHolderArray);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 59f2eb3..ffd23a2 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
@@ -171,11 +171,6 @@ public class CarbonFactDataHandlerModel {
    */
   private boolean isCompactionFlow;
 
-  /**
-   * To use kettle flow to load or not.
-   */
-  private boolean useKettle = true;
-
   private int bucketId = 0;
 
   private String segmentId;
@@ -287,7 +282,6 @@ public class CarbonFactDataHandlerModel {
     carbonFactDataHandlerModel.setCarbonDataDirectoryPath(carbonDataDirectoryPath);
     carbonFactDataHandlerModel.setIsUseInvertedIndex(isUseInvertedIndex);
     carbonFactDataHandlerModel.setBlockSizeInMB(carbonTable.getBlockSizeInMB());
-    carbonFactDataHandlerModel.setUseKettle(false);
     if (noDictionaryCount > 0 || complexDimensionCount > 0) {
       carbonFactDataHandlerModel.setMdKeyIndex(measureCount + 1);
     } else {
@@ -500,14 +494,6 @@ public class CarbonFactDataHandlerModel {
     this.wrapperColumnSchema = wrapperColumnSchema;
   }
 
-  public boolean isUseKettle() {
-    return useKettle;
-  }
-
-  public void setUseKettle(boolean useKettle) {
-    this.useKettle = useKettle;
-  }
-
   public int getBucketId() {
     return bucketId;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
index 8c94328..68f9bd5 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
@@ -95,8 +95,6 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
 
   private char[] aggType;
 
-  private boolean useKettle;
-
   /**
    * below code is to check whether dimension
    * is of no dictionary type or not
@@ -105,7 +103,7 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
 
   public SingleThreadFinalSortFilesMerger(String tempFileLocation, String tableName,
       int dimensionCount, int complexDimensionCount, int measureCount, int noDictionaryCount,
-      char[] aggType, boolean[] isNoDictionaryColumn, boolean useKettle) {
+      char[] aggType, boolean[] isNoDictionaryColumn) {
     this.tempFileLocation = tempFileLocation;
     this.tableName = tableName;
     this.dimensionCount = dimensionCount;
@@ -114,7 +112,6 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
     this.aggType = aggType;
     this.noDictionaryCount = noDictionaryCount;
     this.isNoDictionaryColumn = isNoDictionaryColumn;
-    this.useKettle = useKettle;
   }
 
   /**
@@ -183,8 +180,7 @@ public class SingleThreadFinalSortFilesMerger extends CarbonIterator<Object[]> {
           // create chunk holder
           SortTempFileChunkHolder sortTempFileChunkHolder =
               new SortTempFileChunkHolder(tempFile, dimensionCount, complexDimensionCount,
-                  measureCount, fileBufferSize, noDictionaryCount, aggType, isNoDictionaryColumn,
-                  useKettle);
+                  measureCount, fileBufferSize, noDictionaryCount, aggType, isNoDictionaryColumn);
 
           // initialize
           sortTempFileChunkHolder.initialize();

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

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

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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 a73b356..cda907c 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
@@ -66,7 +66,7 @@ import org.apache.carbondata.core.writer.CarbonIndexFileWriter;
 import org.apache.carbondata.format.BlockIndex;
 import org.apache.carbondata.format.BlockletInfo3;
 import org.apache.carbondata.format.IndexHeader;
-import org.apache.carbondata.processing.mdkeygen.file.FileData;
+import org.apache.carbondata.processing.store.file.FileData;
 import org.apache.carbondata.processing.store.writer.exception.CarbonDataWriterException;
 
 import org.apache.commons.lang3.ArrayUtils;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java
index 9ed0baa..ce53ec8 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/store/writer/CarbonDataWriterVo.java
@@ -20,8 +20,8 @@ import java.util.List;
 
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
-import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.apache.carbondata.processing.store.CarbonDataFileAttributes;
+import org.apache.carbondata.processing.store.file.IFileManagerComposite;
 
 /**
  * Value object for writing the data

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
deleted file mode 100644
index 86b63df..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
+++ /dev/null
@@ -1,495 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.sql.Connection;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-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.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
-import org.apache.carbondata.core.writer.HierarchyValueWriterForCSV;
-import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.mdkeygen.file.IFileManagerComposite;
-import org.apache.carbondata.processing.schema.metadata.ArrayWrapper;
-import org.apache.carbondata.processing.schema.metadata.ColumnSchemaDetails;
-import org.apache.carbondata.processing.schema.metadata.ColumnsInfo;
-
-import org.pentaho.di.core.exception.KettleException;
-
-public abstract class CarbonCSVBasedDimSurrogateKeyGen {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonCSVBasedDimSurrogateKeyGen.class.getName());
-  /**
-   * max
-   */
-  protected int[] max;
-  /**
-   * connection
-   */
-  protected Connection connection;
-  /**
-   * hierInsertFileNames
-   */
-  protected Map<String, String> hierInsertFileNames;
-  /**
-   * dimInsertFileNames
-   */
-  protected String[] dimInsertFileNames;
-  /**
-   * columnsInfo
-   */
-  protected ColumnsInfo columnsInfo;
-  /**
-   * primary key max surrogate key map
-   */
-  protected Map<String, Integer> primaryKeysMaxSurroagetMap;
-  /**
-   * Measure max surrogate key map
-   */
-  protected Map<String, Integer> measureMaxSurroagetMap;
-  /**
-   * File manager
-   */
-  protected IFileManagerComposite fileManager;
-
-  /**
-   * Cache should be map only. because, multiple levels can map to same
-   * database column. This case duplicate storage should be avoided.
-   */
-  private Map<String, Dictionary> dictionaryCaches;
-  /**
-   * Year Cache
-   */
-  private Map<String, Map<String, Integer>> timeDimCache;
-  /**
-   * dimsFiles
-   */
-  private String[] dimsFiles;
-  /**
-   * timeDimMax
-   */
-  private int[] timDimMax;
-  /**
-   * hierCache
-   */
-  private Map<String, Map<Integer, int[]>> hierCache =
-      new HashMap<String, Map<Integer, int[]>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  /**
-   *
-   */
-  private Map<String, Map<ArrayWrapper, Integer>> hierCacheReverse =
-      new HashMap<String, Map<ArrayWrapper, Integer>>(
-          CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  /**
-   * dimension ordinal to dimension mapping
-   */
-  private CarbonDimension[] dimensionOrdinalToDimensionMapping;
-  /**
-   * rwLock2
-   */
-  private ReentrantReadWriteLock rwLock2 = new ReentrantReadWriteLock();
-  /**
-   * wLock2
-   */
-  protected Lock wLock2 = rwLock2.writeLock();
-  /**
-   * Store Folder Name with Load number.
-   */
-  private String storeFolderWithLoadNumber;
-
-  /**
-   * @param columnsInfo ColumnsInfo With all the required details for surrogate key generation and
-   *                    hierarchy entries.
-   */
-  public CarbonCSVBasedDimSurrogateKeyGen(ColumnsInfo columnsInfo) {
-    this.columnsInfo = columnsInfo;
-
-    setDimensionTables(columnsInfo.getDimColNames());
-    setHierFileNames(columnsInfo.getHierTables());
-  }
-
-  /**
-   * @param tuple         The string value whose surrogate key will be gennerated.
-   * @param tabColumnName The K of dictionaryCaches Map, for example "tablename_columnname"
-   */
-  public Integer generateSurrogateKeys(String tuple, String tabColumnName) throws KettleException {
-    Integer key = null;
-    Dictionary dicCache = dictionaryCaches.get(tabColumnName);
-    key = dicCache.getSurrogateKey(tuple);
-    return key;
-  }
-
-  /**
-   * @param tuple         The string value whose surrogate key will be gennerated.
-   * @param tabColumnName The K of dictionaryCaches Map, for example "tablename_columnname"
-   */
-  public Integer generateSurrogateKeys(String tuple, String tabColumnName, String columnId)
-      throws KettleException {
-    Integer key = null;
-    Dictionary dicCache = dictionaryCaches.get(tabColumnName);
-    if (null == dicCache) {
-      ColumnSchemaDetails columnSchemaDetails =
-          this.columnsInfo.getColumnSchemaDetailsWrapper().get(columnId);
-      if (columnSchemaDetails.isDirectDictionary()) {
-        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(columnSchemaDetails.getColumnType());
-        key = directDictionaryGenerator.generateDirectSurrogateKey(tuple);
-      }
-    } else {
-      key = dicCache.getSurrogateKey(tuple);
-    }
-    return key;
-  }
-
-
-  public Integer generateSurrogateKeysForTimeDims(String tuple, String columnName, int index,
-      Object[] props) throws KettleException {
-    Integer key = null;
-    Dictionary dicCache = dictionaryCaches.get(columnName);
-    key = dicCache.getSurrogateKey(tuple);
-    if (key == null) {
-      if (timDimMax[index] >= columnsInfo.getMaxKeys()[index]) {
-        if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(tuple)) {
-          tuple = null;
-        }
-        LOGGER.error("Invalid cardinality. Key size exceeded cardinality for: " + columnsInfo
-                .getDimColNames()[index] + ": MemberValue: " + tuple);
-        return -1;
-      }
-      timDimMax[index]++;
-      Map<String, Integer> timeCache = timeDimCache.get(columnName);
-      // Extract properties from tuple
-      // Need to create a new surrogate key.
-      key = getSurrogateFromStore(tuple, index, props);
-      if (null != timeCache) {
-        timeCache.put(tuple, key);
-      }
-    } else {
-      return updateSurrogateToStore(tuple, columnName, index, key, props);
-    }
-    return key;
-  }
-
-  public void checkNormalizedHierExists(int[] val, String hier,
-      HierarchyValueWriterForCSV hierWriter) throws KettleException {
-    Map<ArrayWrapper, Integer> cache = hierCacheReverse.get(hier);
-
-    ArrayWrapper wrapper = new ArrayWrapper(val);
-    Integer hCache = cache.get(wrapper);
-    if (hCache != null) {
-      return;
-    } else {
-      wLock2.lock();
-      try {
-        getNormalizedHierFromStore(val, hier, 1, hierWriter);
-        // Store in cache
-        cache.put(wrapper, 1);
-      } finally {
-        wLock2.unlock();
-      }
-    }
-  }
-
-  public void close() throws Exception {
-    if (null != connection) {
-      connection.close();
-    }
-  }
-
-  /**
-   * Search entry and insert if not found in store.
-   *
-   * @param val
-   * @param hier
-   * @return
-   * @throws KeyGenException
-   * @throws KettleException
-   */
-  protected abstract byte[] getNormalizedHierFromStore(int[] val, String hier, int primaryKey,
-      HierarchyValueWriterForCSV hierWriter) throws KettleException;
-
-  /**
-   * Search entry and insert if not found in store.
-   *
-   * @param value
-   * @param index
-   * @param properties - Ordinal column, name column and all other properties
-   * @return
-   * @throws KettleException
-   */
-  protected abstract int getSurrogateFromStore(String value, int index, Object[] properties)
-      throws KettleException;
-
-  /**
-   * Search entry and insert if not found in store.
-   *
-   * @param value
-   * @param columnName
-   * @param index
-   * @param properties - Ordinal column, name column and all other properties
-   * @return
-   * @throws KettleException
-   */
-  protected abstract int updateSurrogateToStore(String value, String columnName, int index, int key,
-      Object[] properties) throws KettleException;
-
-  /**
-   * generate the surroagate key for the measure values.
-   *
-   * @return
-   * @throws KettleException
-   */
-  public abstract int getSurrogateForMeasure(String tuple, String columnName)
-      throws KettleException;
-
-  private Map<Integer, int[]> getHCache(String hName) {
-    Map<Integer, int[]> hCache = hierCache.get(hName);
-    if (hCache == null) {
-      hCache = new HashMap<Integer, int[]>();
-      hierCache.put(hName, hCache);
-    }
-
-    return hCache;
-  }
-
-  private Map<ArrayWrapper, Integer> getHCacheReverse(String hName) {
-    Map<ArrayWrapper, Integer> hCache = hierCacheReverse.get(hName);
-    if (hCache == null) {
-      hCache = new HashMap<ArrayWrapper, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-      hierCacheReverse.put(hName, hCache);
-    }
-
-    return hCache;
-  }
-
-  private void setHierFileNames(Set<String> set) {
-    hierInsertFileNames =
-        new HashMap<String, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (String s : set) {
-      hierInsertFileNames.put(s, s + CarbonCommonConstants.HIERARCHY_FILE_EXTENSION);
-
-      // fix hierStream is null issue
-      getHCache(s);
-      getHCacheReverse(s);
-    }
-  }
-
-  private void setDimensionTables(String[] dimeFileNames) {
-    int noOfPrimitiveDims = 0;
-    List<String> dimFilesForPrimitives = new ArrayList<String>();
-    List<Boolean> isDirectDictionary = new ArrayList<Boolean>();
-    dictionaryCaches = new ConcurrentHashMap<String, Dictionary>();
-    for (int i = 0; i < dimeFileNames.length; i++) {
-      GenericDataType complexType = columnsInfo.getComplexTypesMap()
-          .get(dimeFileNames[i].substring(columnsInfo.getTableName().length() + 1));
-      if (complexType != null) {
-        List<GenericDataType> primitiveChild = new ArrayList<GenericDataType>();
-        complexType.getAllPrimitiveChildren(primitiveChild);
-        for (GenericDataType eachPrimitive : primitiveChild) {
-          dimFilesForPrimitives.add(
-              columnsInfo.getTableName() + CarbonCommonConstants.UNDERSCORE + eachPrimitive
-                  .getName());
-          eachPrimitive.setSurrogateIndex(noOfPrimitiveDims);
-          noOfPrimitiveDims++;
-          ColumnSchemaDetails columnSchemaDetails =
-              columnsInfo.getColumnSchemaDetailsWrapper().get(eachPrimitive.getColumnId());
-          if (columnSchemaDetails.isDirectDictionary()) {
-            isDirectDictionary.add(true);
-          } else {
-            isDirectDictionary.add(false);
-          }
-        }
-      } else {
-        dimFilesForPrimitives.add(dimeFileNames[i]);
-        noOfPrimitiveDims++;
-        isDirectDictionary.add(false);
-      }
-    }
-    max = new int[noOfPrimitiveDims];
-    for (int i = 0; i < isDirectDictionary.size(); i++) {
-      if (isDirectDictionary.get(i)) {
-        max[i] = Integer.MAX_VALUE;
-      }
-    }
-    this.dimsFiles = dimFilesForPrimitives.toArray(new String[dimFilesForPrimitives.size()]);
-
-    createRespectiveDimFilesForDimTables();
-  }
-
-  private void createRespectiveDimFilesForDimTables() {
-    int dimCount = this.dimsFiles.length;
-    dimInsertFileNames = new String[dimCount];
-    System.arraycopy(dimsFiles, 0, dimInsertFileNames, 0, dimCount);
-  }
-
-  /**
-   * isCacheFilled
-   *
-   * @param columnNames
-   * @return boolean
-   */
-  public abstract boolean isCacheFilled(String[] columnNames);
-
-  /**
-   * @return Returns the storeFolderWithLoadNumber.
-   */
-  public String getStoreFolderWithLoadNumber() {
-    return storeFolderWithLoadNumber;
-  }
-
-  /**
-   * @param storeFolderWithLoadNumber The storeFolderWithLoadNumber to set.
-   */
-  public void setStoreFolderWithLoadNumber(String storeFolderWithLoadNumber) {
-    this.storeFolderWithLoadNumber = storeFolderWithLoadNumber;
-  }
-
-  /**
-   * @return Returns the dictionaryCaches.
-   */
-  public Map<String, Dictionary> getDictionaryCaches() {
-    return dictionaryCaches;
-  }
-
-  /**
-   * @param dictionaryCaches The dictionaryCaches to set.
-   */
-  public void setDictionaryCaches(Map<String, Dictionary> dictionaryCaches) {
-    this.dictionaryCaches = dictionaryCaches;
-  }
-
-  /**
-   * @return Returns the timeDimCache.
-   */
-  public Map<String, Map<String, Integer>> getTimeDimCache() {
-    return timeDimCache;
-  }
-
-  /**
-   * @param timeDimCache The timeDimCache to set.
-   */
-  public void setTimeDimCache(Map<String, Map<String, Integer>> timeDimCache) {
-    this.timeDimCache = timeDimCache;
-  }
-
-  /**
-   * @return Returns the dimsFiles.
-   */
-  public String[] getDimsFiles() {
-    return dimsFiles;
-  }
-
-  /**
-   * @param dimsFiles The dimsFiles to set.
-   */
-  public void setDimsFiles(String[] dimsFiles) {
-    this.dimsFiles = dimsFiles;
-  }
-
-  /**
-   * @return Returns the hierCache.
-   */
-  public Map<String, Map<Integer, int[]>> getHierCache() {
-    return hierCache;
-  }
-
-  /**
-   * @param hierCache The hierCache to set.
-   */
-  public void setHierCache(Map<String, Map<Integer, int[]>> hierCache) {
-    this.hierCache = hierCache;
-  }
-
-  /**
-   * @return Returns the timDimMax.
-   */
-  public int[] getTimDimMax() {
-    return timDimMax;
-  }
-
-  /**
-   * @param timDimMax The timDimMax to set.
-   */
-  public void setTimDimMax(int[] timDimMax) {
-    this.timDimMax = timDimMax;
-  }
-
-  /**
-   * @return the hierCacheReverse
-   */
-  public Map<String, Map<ArrayWrapper, Integer>> getHierCacheReverse() {
-    return hierCacheReverse;
-  }
-
-  /**
-   * @param hierCacheReverse the hierCacheReverse to set
-   */
-  public void setHierCacheReverse(Map<String, Map<ArrayWrapper, Integer>> hierCacheReverse) {
-    this.hierCacheReverse = hierCacheReverse;
-  }
-
-  public int[] getMax() {
-    return max;
-  }
-
-  public void setMax(int[] max) {
-    this.max = max;
-  }
-
-  /**
-   * @return the measureMaxSurroagetMap
-   */
-  public Map<String, Integer> getMeasureMaxSurroagetMap() {
-    return measureMaxSurroagetMap;
-  }
-
-  /**
-   * @param measureMaxSurroagetMap the measureMaxSurroagetMap to set
-   */
-  public void setMeasureMaxSurroagetMap(Map<String, Integer> measureMaxSurroagetMap) {
-    this.measureMaxSurroagetMap = measureMaxSurroagetMap;
-  }
-
-  /**
-   * @return
-   */
-  public CarbonDimension[] getDimensionOrdinalToDimensionMapping() {
-    return dimensionOrdinalToDimensionMapping;
-  }
-
-  /**
-   * @param dimensionOrdinalToDimensionMapping
-   */
-  public void setDimensionOrdinalToDimensionMapping(
-      CarbonDimension[] dimensionOrdinalToDimensionMapping) {
-    this.dimensionOrdinalToDimensionMapping = dimensionOrdinalToDimensionMapping;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenData.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenData.java b/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenData.java
deleted file mode 100644
index d78dc1a..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenData.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.HashMap;
-import java.util.Map;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.keygenerator.KeyGenerator;
-
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.trans.step.BaseStepData;
-import org.pentaho.di.trans.step.StepDataInterface;
-
-public class CarbonCSVBasedSeqGenData extends BaseStepData implements StepDataInterface {
-
-  /**
-   * outputRowMeta
-   */
-  private RowMetaInterface outputRowMeta;
-
-  /**
-   * surrogateKeyGen
-   */
-  private CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen;
-
-  /**
-   * keyGenerators
-   */
-  private Map<String, KeyGenerator> keyGenerators =
-      new HashMap<String, KeyGenerator>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  /**
-   * columnIndex
-   */
-  private Map<String, int[]> columnIndex =
-      new HashMap<String, int[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  /**
-   * precomputed default objects
-   */
-  private Object[] defaultObjects;
-
-  /**
-   * generator
-   */
-  private KeyGenerator generator;
-
-  /**
-   * the size of the input rows
-   */
-  private int inputSize;
-
-
-  public CarbonCSVBasedSeqGenData() {
-    super();
-  }
-
-  /**
-   * @return Returns the surrogateKeyGen.
-   */
-  public CarbonCSVBasedDimSurrogateKeyGen getSurrogateKeyGen() {
-    return surrogateKeyGen;
-  }
-
-  /**
-   * @param surrogateKeyGen The surrogateKeyGen to set.
-   */
-  public void setSurrogateKeyGen(CarbonCSVBasedDimSurrogateKeyGen surrogateKeyGen) {
-    this.surrogateKeyGen = surrogateKeyGen;
-  }
-
-  /**
-   * @param inputSize The inputSize to set.
-   */
-  public void setInputSize(int inputSize) {
-    this.inputSize = inputSize;
-  }
-
-  /**
-   * @param generator The generator to set.
-   */
-  public void setGenerator(KeyGenerator generator) {
-    this.generator = generator;
-  }
-
-  /**
-   * @return Returns the keyGenerators.
-   */
-  public Map<String, KeyGenerator> getKeyGenerators() {
-    return keyGenerators;
-  }
-
-  /**
-   * @return Returns the outputRowMeta.
-   */
-  public RowMetaInterface getOutputRowMeta() {
-    return outputRowMeta;
-  }
-
-  /**
-   * @param outputRowMeta The outputRowMeta to set.
-   */
-  public void setOutputRowMeta(RowMetaInterface outputRowMeta) {
-    this.outputRowMeta = outputRowMeta;
-  }
-
-  public void clean() {
-    outputRowMeta = null;
-
-    surrogateKeyGen = null;
-
-    generator = null;
-    keyGenerators = null;
-
-    columnIndex = null;
-
-    defaultObjects = null;
-
-  }
-}


[11/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 85e3cd7..cc16398 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
@@ -67,13 +67,6 @@ 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.api.dataloader.DataLoadModel;
-import org.apache.carbondata.processing.api.dataloader.SchemaInfo;
-import org.apache.carbondata.processing.csvload.DataGraphExecuter;
-import org.apache.carbondata.processing.dataprocessor.DataProcessTaskStatus;
-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.spark.merger.NodeBlockRelation;
 import org.apache.carbondata.spark.merger.NodeMultiBlockRelation;
@@ -90,97 +83,7 @@ public final class CarbonLoaderUtil {
   private CarbonLoaderUtil() {
   }
 
-  private static void generateGraph(IDataProcessStatus dataProcessTaskStatus, SchemaInfo info,
-      CarbonLoadModel loadModel, String outputLocation)
-      throws GraphGeneratorException {
-    DataLoadModel model = new DataLoadModel();
-    model.setCsvLoad(null != dataProcessTaskStatus.getCsvFilePath()
-            || null != dataProcessTaskStatus.getFilesToProcess());
-    model.setSchemaInfo(info);
-    model.setTableName(dataProcessTaskStatus.getTableName());
-    List<LoadMetadataDetails> loadMetadataDetails = loadModel.getLoadMetadataDetails();
-    model.setBlocksID(dataProcessTaskStatus.getBlocksID());
-    model.setEscapeCharacter(dataProcessTaskStatus.getEscapeCharacter());
-    model.setQuoteCharacter(dataProcessTaskStatus.getQuoteCharacter());
-    model.setCommentCharacter(dataProcessTaskStatus.getCommentCharacter());
-    model.setRddIteratorKey(dataProcessTaskStatus.getRddIteratorKey());
-    model.setTaskNo(loadModel.getTaskNo());
-    model.setFactTimeStamp(loadModel.getFactTimeStamp());
-    model.setMaxColumns(loadModel.getMaxColumns());
-    model.setDateFormat(loadModel.getDateFormat());
-    GraphGenerator generator = new GraphGenerator(model, loadModel.getPartitionId(),
-        loadModel.getStorePath(), loadModel.getCarbonDataLoadSchema(), loadModel.getSegmentId(),
-        outputLocation);
-    generator.generateGraph();
-  }
 
-  public static void executeGraph(CarbonLoadModel loadModel, String storeLocation,
-      String storePath, String kettleHomePath) throws Exception {
-    System.setProperty("KETTLE_HOME", kettleHomePath);
-    if (!new File(storeLocation).mkdirs()) {
-      LOGGER.error("Error while creating the temp store path: " + storeLocation);
-    }
-    String outPutLoc = storeLocation + "/etl";
-    String databaseName = loadModel.getDatabaseName();
-    String tableName = loadModel.getTableName();
-    String tempLocationKey = databaseName + CarbonCommonConstants.UNDERSCORE + tableName
-        + CarbonCommonConstants.UNDERSCORE + loadModel.getTaskNo();
-    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 = "";
-    if (loadModel.isAggLoadRequest()) {
-      fileNamePrefix = "graphgenerator";
-    }
-    String graphPath =
-        outPutLoc + File.separator + databaseName + File.separator + tableName + File.separator
-            + loadModel.getSegmentId() + File.separator + loadModel.getTaskNo() + File.separator
-            + tableName + fileNamePrefix + ".ktr";
-    File path = new File(graphPath);
-    if (path.exists()) {
-      path.delete();
-    }
-
-    DataProcessTaskStatus dataProcessTaskStatus
-            = new DataProcessTaskStatus(databaseName, tableName);
-    dataProcessTaskStatus.setCsvFilePath(loadModel.getFactFilePath());
-    if (loadModel.isDirectLoad()) {
-      dataProcessTaskStatus.setFilesToProcess(loadModel.getFactFilesToProcess());
-      dataProcessTaskStatus.setDirectLoad(true);
-      dataProcessTaskStatus.setCsvDelimiter(loadModel.getCsvDelimiter());
-      dataProcessTaskStatus.setCsvHeader(loadModel.getCsvHeader());
-    }
-
-    dataProcessTaskStatus.setBlocksID(loadModel.getBlocksID());
-    dataProcessTaskStatus.setEscapeCharacter(loadModel.getEscapeChar());
-    dataProcessTaskStatus.setQuoteCharacter(loadModel.getQuoteChar());
-    dataProcessTaskStatus.setCommentCharacter(loadModel.getCommentChar());
-    dataProcessTaskStatus.setRddIteratorKey(loadModel.getRddIteratorKey());
-    dataProcessTaskStatus.setDateFormat(loadModel.getDateFormat());
-    SchemaInfo info = new SchemaInfo();
-    info.setDatabaseName(databaseName);
-    info.setTableName(tableName);
-    info.setAutoAggregateRequest(loadModel.isAggLoadRequest());
-    info.setComplexDelimiterLevel1(loadModel.getComplexDelimiterLevel1());
-    info.setComplexDelimiterLevel2(loadModel.getComplexDelimiterLevel2());
-    info.setSerializationNullFormat(loadModel.getSerializationNullFormat());
-    info.setBadRecordsLoggerEnable(loadModel.getBadRecordsLoggerEnable());
-    info.setBadRecordsLoggerAction(loadModel.getBadRecordsAction());
-
-    generateGraph(dataProcessTaskStatus, info, loadModel, outPutLoc);
-
-    DataGraphExecuter graphExecuter = new DataGraphExecuter(dataProcessTaskStatus);
-    graphExecuter
-        .executeGraph(graphPath, info, loadModel.getCarbonDataLoadSchema());
-  }
-
-  public static List<String> addNewSliceNameToList(String newSlice, List<String> activeSlices) {
-    activeSlices.add(newSlice);
-    return activeSlices;
-  }
 
   public static void deleteSegment(CarbonLoadModel loadModel, int currentLoad) {
     CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
@@ -252,43 +155,6 @@ public final class CarbonLoaderUtil {
     }
   }
 
-  public static List<String> getListOfValidSlices(LoadMetadataDetails[] details) {
-    List<String> activeSlices =
-        new ArrayList<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (LoadMetadataDetails oneLoad : details) {
-      if (CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS.equals(oneLoad.getLoadStatus())
-          || CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS.equals(oneLoad.getLoadStatus())
-          || CarbonCommonConstants.MARKED_FOR_UPDATE.equals(oneLoad.getLoadStatus())) {
-        if (null != oneLoad.getMergedLoadName()) {
-          String loadName = CarbonCommonConstants.LOAD_FOLDER + oneLoad.getMergedLoadName();
-          activeSlices.add(loadName);
-        } else {
-          String loadName = CarbonCommonConstants.LOAD_FOLDER + oneLoad.getLoadName();
-          activeSlices.add(loadName);
-        }
-      }
-    }
-    return activeSlices;
-  }
-
-  public static List<String> getListOfUpdatedSlices(LoadMetadataDetails[] details) {
-    List<String> updatedSlices =
-        new ArrayList<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (LoadMetadataDetails oneLoad : details) {
-      if (CarbonCommonConstants.MARKED_FOR_UPDATE.equals(oneLoad.getLoadStatus())) {
-        if (null != oneLoad.getMergedLoadName()) {
-          updatedSlices.add(oneLoad.getMergedLoadName());
-        } else {
-          updatedSlices.add(oneLoad.getLoadName());
-        }
-      }
-    }
-    return updatedSlices;
-  }
-
-  public static void removeSliceFromMemory(String databaseName, String tableName, String loadName) {
-    // TODO: Remove from memory
-  }
 
   /**
    * This method will delete the local data load folder location after data load is complete

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java
index dd41c59..29aa7e7 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/RowResultMerger.java
@@ -46,13 +46,13 @@ import org.apache.carbondata.core.util.DataTypeUtil;
 import org.apache.carbondata.core.util.path.CarbonStorePath;
 import org.apache.carbondata.core.util.path.CarbonTablePath;
 import org.apache.carbondata.processing.datatypes.GenericDataType;
-import org.apache.carbondata.processing.merger.exeception.SliceMergerException;
 import org.apache.carbondata.processing.model.CarbonLoadModel;
 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.store.writer.exception.CarbonDataWriterException;
+import org.apache.carbondata.spark.merger.exeception.SliceMergerException;
 
 /**
  * This is the Merger class responsible for the merging of the segments.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java
index c65dab6..08b563f 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/TupleConversionAdapter.java
@@ -16,12 +16,8 @@
  */
 package org.apache.carbondata.spark.merger;
 
-import java.util.ArrayList;
-import java.util.List;
-
 import org.apache.carbondata.core.datastore.block.SegmentProperties;
 import org.apache.carbondata.core.scan.wrappers.ByteArrayWrapper;
-import org.apache.carbondata.processing.util.NonDictionaryUtil;
 
 /**
  * This class will be used to convert the Result into the format used in data writer.
@@ -61,15 +57,7 @@ class TupleConversionAdapter {
 
     // put No dictionary byte []
     if (isNoDictionaryPresent) {
-
-      int noDicCount = segmentproperties.getNumberOfNoDictionaryDimension();
-      List<byte[]> noDicByteArr = new ArrayList<>(noDicCount);
-      for (int i = 0; i < noDicCount; i++) {
-        noDicByteArr.add(((ByteArrayWrapper) carbonTuple[0]).getNoDictionaryKeyByIndex(i));
-      }
-      byte[] singleByteArr = NonDictionaryUtil.convertListByteArrToSingleArr(noDicByteArr);
-
-      row[index++] = singleByteArr;
+      row[index++] = ((ByteArrayWrapper) carbonTuple[0]).getNoDictionaryKeys();
     }
 
     // put No Dictionary Dims

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/exeception/SliceMergerException.java
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/exeception/SliceMergerException.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/exeception/SliceMergerException.java
new file mode 100644
index 0000000..fd6610c
--- /dev/null
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/merger/exeception/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.spark.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/incubator-carbondata/blob/e6b60907/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
index ae8f784..297bd7d 100644
--- 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
@@ -16,12 +16,7 @@
  */
 
 /**
- * Project Name  : Carbon
- * Module Name   : CARBON Data Processor
- * Created Date  : 15-Sep-2015
- * FileName   : LoadMetadataUtil.java
- * Description   : Kettle step to generate MD Key
- * Class Version  : 1.0
+ * Utility for load data
  */
 package org.apache.carbondata.spark.util;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 f8f11d6..c29c1a2 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
@@ -44,8 +44,6 @@ class CarbonOption(options: Map[String, String]) {
 
   def compress: Boolean = options.getOrElse("compress", "false").toBoolean
 
-  def useKettle: Boolean = options.getOrElse("use_kettle", "false").toBoolean
-
   def singlePass: Boolean = options.getOrElse("single_pass", "false").toBoolean
 
   def dictionaryInclude: Option[String] = options.get("dictionary_include")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
deleted file mode 100644
index e3f2cf6..0000000
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
+++ /dev/null
@@ -1,754 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF 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.rdd
-
-import java.lang.Long
-import java.nio.ByteBuffer
-import java.text.SimpleDateFormat
-import java.util
-import java.util.UUID
-
-import scala.collection.JavaConverters._
-import scala.util.Random
-
-import org.apache.spark.{Partition, SerializableWritable, SparkContext, SparkEnv, TaskContext}
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionWrap, RDD}
-import org.apache.spark.sql.Row
-import org.apache.spark.sql.execution.command.ExecutionErrors
-import org.apache.spark.util.SparkUtil
-
-import org.apache.carbondata.common.CarbonIterator
-import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.common.logging.impl.StandardLogService
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory}
-import org.apache.carbondata.processing.constants.DataProcessorConstants
-import org.apache.carbondata.processing.csvreaderstep.{BlockDetails, RddInputUtils}
-import org.apache.carbondata.processing.etl.DataLoadingException
-import org.apache.carbondata.processing.graphgenerator.GraphGenerator
-import org.apache.carbondata.processing.model.CarbonLoadModel
-import org.apache.carbondata.spark.DataLoadResult
-import org.apache.carbondata.spark.load._
-import org.apache.carbondata.spark.splits.TableSplit
-import org.apache.carbondata.spark.util.{CarbonQueryUtil, CarbonScalaUtil}
-
-/**
- * This partition class use to split by TableSplit
- *
- */
-class CarbonTableSplitPartition(rddId: Int, val idx: Int, @transient val tableSplit: TableSplit,
-    val blocksDetails: Array[BlockDetails])
-  extends Partition {
-
-  override val index: Int = idx
-  val serializableHadoopSplit = new SerializableWritable[TableSplit](tableSplit)
-  val partitionBlocksDetail = blocksDetails
-
-  override def hashCode(): Int = 41 * (41 + rddId) + idx
-}
-
-/**
- * This partition class use to split by Host
- *
- */
-class CarbonNodePartition(rddId: Int, val idx: Int, host: String,
-    val blocksDetails: Array[BlockDetails])
-  extends Partition {
-
-  override val index: Int = idx
-  val serializableHadoopSplit = host
-  val nodeBlocksDetail = blocksDetails
-
-  override def hashCode(): Int = 41 * (41 + rddId) + idx
-}
-
-class SparkPartitionLoader(model: CarbonLoadModel,
-    splitIndex: Int,
-    storePath: String,
-    kettleHomePath: String,
-    loadCount: String,
-    loadMetadataDetails: LoadMetadataDetails) {
-  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-
-  var storeLocation: String = ""
-
-  def initialize(): Unit = {
-    val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null)
-    if (null == carbonPropertiesFilePath) {
-      System.setProperty("carbon.properties.filepath",
-        System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties")
-    }
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance.initPartitonInfo(model.getPartitionId)
-    CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true")
-    CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1")
-    CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true")
-    CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true")
-    CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true")
-    CarbonProperties.getInstance().addProperty("high.cardinality.value", "100000")
-    CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false")
-    CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000")
-
-    // this property is used to determine whether temp location for carbon is inside
-    // container temp dir or is yarn application directory.
-    val carbonUseLocalDir = CarbonProperties.getInstance()
-      .getProperty("carbon.use.local.dir", "false")
-    if (carbonUseLocalDir.equalsIgnoreCase("true")) {
-      val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
-      if (null != storeLocations && storeLocations.nonEmpty) {
-        storeLocation = storeLocations(Random.nextInt(storeLocations.length))
-      }
-      if (storeLocation == null) {
-        storeLocation = System.getProperty("java.io.tmpdir")
-      }
-    } else {
-      storeLocation = System.getProperty("java.io.tmpdir")
-    }
-    storeLocation = storeLocation + '/' + System.nanoTime() + '/' + splitIndex
-  }
-
-  def run(): Unit = {
-    try {
-      CarbonLoaderUtil.executeGraph(model, storeLocation, storePath,
-        kettleHomePath)
-      loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)
-    } catch {
-      case e: DataLoadingException => if (e.getErrorCode ==
-                                          DataProcessorConstants.BAD_REC_FOUND) {
-        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)
-        LOGGER.info("Bad Record Found")
-      } else {
-        throw e
-      }
-      case e: Exception =>
-        throw e
-    } finally {
-      // delete temp location data
-      try {
-        val isCompaction = false
-        CarbonLoaderUtil.deleteLocalDataLoadFolderLocation(model, isCompaction)
-      } catch {
-        case e: Exception =>
-          LOGGER.error(e, "Failed to delete local data")
-      }
-      if (!CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals(
-        loadMetadataDetails.getLoadStatus)) {
-        if (CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
-          .equals(loadMetadataDetails.getLoadStatus)) {
-          LOGGER.info("DataLoad complete")
-          LOGGER.info("Data Load partially successful with LoadCount:" + loadCount)
-        } else {
-          LOGGER.info("DataLoad complete")
-          LOGGER.info("Data Loaded successfully with LoadCount:" + loadCount)
-          CarbonTimeStatisticsFactory.getLoadStatisticsInstance.printStatisticsInfo(
-            model.getPartitionId)
-        }
-      }
-    }
-  }
-}
-
-/**
- * Use this RDD class to load csv data file
- *
- * @param sc                    The SparkContext to associate the RDD with.
- * @param result                Output result
- * @param carbonLoadModel       Carbon load model which contain the load info
- * @param storePath             The store location
- * @param kettleHomePath        The kettle home path
- * @param columnar             whether it is columnar
- * @param loadCount             Current load count
- * @param tableCreationTime     Time of creating table
- * @param schemaLastUpdatedTime Time of last schema update
- * @param blocksGroupBy         Blocks Array which is group by partition or host
- * @param isTableSplitPartition Whether using table split partition
- * @tparam K Class of the key associated with the Result.
- * @tparam V Class of the value associated with the Result.
- */
-class DataFileLoaderRDD[K, V](
-    sc: SparkContext,
-    result: DataLoadResult[K, V],
-    carbonLoadModel: CarbonLoadModel,
-    storePath: String,
-    kettleHomePath: String,
-    columnar: Boolean,
-    loadCount: Integer,
-    tableCreationTime: Long,
-    schemaLastUpdatedTime: Long,
-    blocksGroupBy: Array[(String, Array[BlockDetails])],
-    isTableSplitPartition: Boolean) extends RDD[(K, V)](sc, Nil) {
-
-  sc.setLocalProperty("spark.scheduler.pool", "DDL")
-
-  override def getPartitions: Array[Partition] = {
-    if (isTableSplitPartition) {
-      // for table split partition
-      var splits = Array[TableSplit]()
-      if (carbonLoadModel.isDirectLoad) {
-        splits = CarbonQueryUtil.getTableSplitsForDirectLoad(carbonLoadModel.getFactFilePath)
-      } else {
-        splits = CarbonQueryUtil.getTableSplits(carbonLoadModel.getDatabaseName,
-          carbonLoadModel.getTableName, null)
-      }
-
-      splits.zipWithIndex.map { case (split, index) =>
-        // filter the same partition unique id, because only one will match, so get 0 element
-        val blocksDetails: Array[BlockDetails] = blocksGroupBy.filter { case (uniqueId, _) =>
-          uniqueId == split.getPartition.getUniqueID
-        }(0)._2
-        new CarbonTableSplitPartition(id, index, split, blocksDetails)
-      }
-    } else {
-      // for node partition
-      blocksGroupBy.zipWithIndex.map { case ((uniqueId, blockDetails), index) =>
-        new CarbonNodePartition(id, index, uniqueId, blockDetails)
-      }
-    }
-  }
-
-  override def checkpoint() {
-    // Do nothing. Hadoop RDD should not be checkpointed.
-  }
-
-  override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val iter = new Iterator[(K, V)] {
-      var partitionID = "0"
-      val loadMetadataDetails = new LoadMetadataDetails()
-      var model: CarbonLoadModel = _
-      val uniqueLoadStatusId = carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE +
-          theSplit.index
-      try {
-        loadMetadataDetails.setPartitionCount(partitionID)
-        carbonLoadModel.setSegmentId(String.valueOf(loadCount))
-        setModelAndBlocksInfo()
-        val loader = new SparkPartitionLoader(model, theSplit.index, storePath,
-          kettleHomePath, String.valueOf(loadCount), loadMetadataDetails)
-        loader.initialize
-        if (model.isRetentionRequest) {
-          recreateAggregationTableForRetention
-        } else if (model.isAggLoadRequest) {
-          loadMetadataDetails.setLoadStatus(createManualAggregateTable)
-        } else {
-          loader.run()
-        }
-      } catch {
-        case e: Exception =>
-          logInfo("DataLoad failure")
-          LOGGER.error(e)
-          throw e
-      }
-
-      def setModelAndBlocksInfo(): Unit = {
-        if (isTableSplitPartition) {
-          // for table split partition
-          val split = theSplit.asInstanceOf[CarbonTableSplitPartition]
-          logInfo("Input split: " + split.serializableHadoopSplit.value)
-          val blocksID = gernerateBlocksID
-          carbonLoadModel.setBlocksID(blocksID)
-          carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
-          if (carbonLoadModel.isDirectLoad) {
-            model = carbonLoadModel.getCopyWithPartition(
-              split.serializableHadoopSplit.value.getPartition.getUniqueID,
-              split.serializableHadoopSplit.value.getPartition.getFilesPath,
-              carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
-          } else {
-            model = carbonLoadModel.getCopyWithPartition(
-              split.serializableHadoopSplit.value.getPartition.getUniqueID)
-          }
-          partitionID = split.serializableHadoopSplit.value.getPartition.getUniqueID
-          // get this partition data blocks and put it to global static map
-          GraphGenerator.blockInfo.put(blocksID, split.partitionBlocksDetail)
-          StandardLogService.setThreadName(partitionID, null)
-          CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordPartitionBlockMap(
-            partitionID, split.partitionBlocksDetail.length)
-        } else {
-          // for node partition
-          val split = theSplit.asInstanceOf[CarbonNodePartition]
-          logInfo("Input split: " + split.serializableHadoopSplit)
-          logInfo("The Block Count in this node: " + split.nodeBlocksDetail.length)
-          CarbonTimeStatisticsFactory.getLoadStatisticsInstance.recordHostBlockMap(
-            split.serializableHadoopSplit, split.nodeBlocksDetail.length)
-          val blocksID = gernerateBlocksID
-          carbonLoadModel.setBlocksID(blocksID)
-          carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
-          // set this node blocks info to global static map
-          GraphGenerator.blockInfo.put(blocksID, split.nodeBlocksDetail)
-          if (carbonLoadModel.isDirectLoad) {
-            val filelist: java.util.List[String] = new java.util.ArrayList[String](
-              CarbonCommonConstants.CONSTANT_SIZE_TEN)
-            CarbonQueryUtil.splitFilePath(carbonLoadModel.getFactFilePath, filelist, ",")
-            model = carbonLoadModel.getCopyWithPartition(partitionID, filelist,
-              carbonLoadModel.getCsvHeader, carbonLoadModel.getCsvDelimiter)
-          } else {
-            model = carbonLoadModel.getCopyWithPartition(partitionID)
-          }
-          StandardLogService.setThreadName(blocksID, null)
-        }
-      }
-
-      /**
-       * generate blocks id
-       *
-       * @return
-       */
-      def gernerateBlocksID: String = {
-        if (isTableSplitPartition) {
-          carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" +
-          theSplit.asInstanceOf[CarbonTableSplitPartition].serializableHadoopSplit.value
-            .getPartition.getUniqueID + "_" + UUID.randomUUID()
-        } else {
-          carbonLoadModel.getDatabaseName + "_" + carbonLoadModel.getTableName + "_" +
-          UUID.randomUUID()
-        }
-      }
-
-      def checkAndLoadAggregationTable: String = {
-        val schema = model.getCarbonDataLoadSchema
-        val aggTables = schema.getCarbonTable.getAggregateTablesName
-        if (null != aggTables && !aggTables.isEmpty) {
-          val details = model.getLoadMetadataDetails.asScala.toArray
-          val newSlice = CarbonCommonConstants.LOAD_FOLDER + loadCount
-          var listOfLoadFolders = CarbonLoaderUtil.getListOfValidSlices(details)
-          listOfLoadFolders = CarbonLoaderUtil.addNewSliceNameToList(newSlice, listOfLoadFolders)
-          val listOfUpdatedLoadFolders = CarbonLoaderUtil.getListOfUpdatedSlices(details)
-          var listOfAllLoadFolders = CarbonQueryUtil.getListOfSlices(details)
-          listOfAllLoadFolders = CarbonLoaderUtil
-            .addNewSliceNameToList(newSlice, listOfAllLoadFolders)
-          val copyListOfLoadFolders = listOfLoadFolders.asScala.toList
-          val copyListOfUpdatedLoadFolders = listOfUpdatedLoadFolders.asScala.toList
-          loadTableSlices(listOfAllLoadFolders, details)
-          val loadFolders = Array[String]()
-          loadMetadataDetails.setLoadStatus(iterateOverAggTables(aggTables,
-            copyListOfLoadFolders.asJava, copyListOfUpdatedLoadFolders.asJava, loadFolders))
-          if (CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals(
-            loadMetadataDetails.getLoadStatus)) {
-            // remove the current slice from memory not the table
-            CarbonLoaderUtil
-              .removeSliceFromMemory(model.getDatabaseName, model.getTableName, newSlice)
-            logInfo(s"Aggregate table creation failed")
-          } else {
-            logInfo("Aggregate tables creation successfull")
-          }
-        }
-        loadMetadataDetails.getLoadStatus
-      }
-
-      def loadTableSlices(listOfAllLoadFolders: java.util.List[String],
-          loadMetadataDetails: Array[LoadMetadataDetails]) = {
-        CarbonProperties.getInstance().addProperty("carbon.cache.used", "false")
-        // TODO: Implement it
-      }
-
-      def createManualAggregateTable: String = {
-        val details = model.getLoadMetadataDetails.asScala.toArray
-        val listOfAllLoadFolders = CarbonQueryUtil.getListOfSlices(details)
-        val listOfLoadFolders = CarbonLoaderUtil.getListOfValidSlices(details)
-        val listOfUpdatedLoadFolders = CarbonLoaderUtil.getListOfUpdatedSlices(details)
-        loadTableSlices(listOfAllLoadFolders, details)
-        val loadFolders = Array[String]()
-        val aggTable = model.getAggTableName
-        loadMetadataDetails.setLoadStatus(loadAggregationTable(listOfLoadFolders,
-          listOfUpdatedLoadFolders, loadFolders))
-        if (CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals(
-          loadMetadataDetails.getLoadStatus)) {
-          logInfo(s"Aggregate table creation failed :: $aggTable")
-        } else {
-          logInfo(s"Aggregate table creation successfull :: $aggTable")
-        }
-        loadMetadataDetails.getLoadStatus
-      }
-
-      def recreateAggregationTableForRetention = {
-        val schema = model.getCarbonDataLoadSchema
-        val aggTables = schema.getCarbonTable.getAggregateTablesName
-        if (null != aggTables && !aggTables.isEmpty) {
-          val details = model.getLoadMetadataDetails.asScala.toArray
-          val listOfLoadFolders = CarbonLoaderUtil.getListOfValidSlices(details)
-          val listOfUpdatedLoadFolders = CarbonLoaderUtil.getListOfUpdatedSlices(details)
-          val listOfAllLoadFolder = CarbonQueryUtil.getListOfSlices(details)
-          loadTableSlices(listOfAllLoadFolder, details)
-          val loadFolders = Array[String]()
-          iterateOverAggTables(aggTables, listOfLoadFolders, listOfUpdatedLoadFolders, loadFolders)
-        }
-      }
-
-      // TODO Aggregate table needs to be handled
-      def iterateOverAggTables(aggTables: java.util.List[String],
-          listOfLoadFolders: java.util.List[String],
-          listOfUpdatedLoadFolders: java.util.List[String],
-          loadFolders: Array[String]): String = {
-        model.setAggLoadRequest(true)
-        aggTables.asScala.foreach { aggTable =>
-          model.setAggTableName(aggTable)
-          loadMetadataDetails.setLoadStatus(loadAggregationTable(listOfLoadFolders,
-            listOfUpdatedLoadFolders, loadFolders))
-          if (CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals(
-            loadMetadataDetails.getLoadStatus)) {
-            logInfo(s"Aggregate table creation failed :: aggTable")
-            return loadMetadataDetails.getLoadStatus
-          }
-        }
-        loadMetadataDetails.getLoadStatus
-      }
-
-      def loadAggregationTable(listOfLoadFolders: java.util.List[String],
-          listOfUpdatedLoadFolders: java.util.List[String],
-          loadFolders: Array[String]): String = {
-        // TODO: Implement it
-        loadMetadataDetails.getLoadStatus
-      }
-
-      var finished = false
-
-      override def hasNext: Boolean = {
-        !finished
-      }
-
-      override def next(): (K, V) = {
-        finished = true
-        result.getKey(uniqueLoadStatusId, loadMetadataDetails)
-      }
-    }
-    iter
-  }
-
-  override def getPreferredLocations(split: Partition): Seq[String] = {
-    if (isTableSplitPartition) {
-      // for table split partition
-      val theSplit = split.asInstanceOf[CarbonTableSplitPartition]
-      val location = theSplit.serializableHadoopSplit.value.getLocations.asScala
-      location
-    } else {
-      // for node partition
-      val theSplit = split.asInstanceOf[CarbonNodePartition]
-      val firstOptionLocation: Seq[String] = List(theSplit.serializableHadoopSplit)
-      logInfo("Preferred Location for split: " + firstOptionLocation.head)
-      val blockMap = new util.LinkedHashMap[String, Integer]()
-      val tableBlocks = theSplit.blocksDetails
-      tableBlocks.foreach { tableBlock =>
-        tableBlock.getLocations.foreach { location =>
-          if (!firstOptionLocation.exists(location.equalsIgnoreCase)) {
-            val currentCount = blockMap.get(location)
-            if (currentCount == null) {
-              blockMap.put(location, 1)
-            } else {
-              blockMap.put(location, currentCount + 1)
-            }
-          }
-        }
-      }
-
-      val sortedList = blockMap.entrySet().asScala.toSeq.sortWith((nodeCount1, nodeCount2) => {
-        nodeCount1.getValue > nodeCount2.getValue
-      }
-      )
-
-      val sortedNodesList = sortedList.map(nodeCount => nodeCount.getKey).take(2)
-      firstOptionLocation ++ sortedNodesList
-    }
-  }
-
-}
-
-/**
- * Use this RDD class to load RDD
- *
- * @param sc
- * @param result
- * @param carbonLoadModel
- * @param storePath
- * @param kettleHomePath
- * @param columnar
- * @param loadCount
- * @param tableCreationTime
- * @param schemaLastUpdatedTime
- * @param prev
- * @tparam K
- * @tparam V
- */
-class DataFrameLoaderRDD[K, V](
-    sc: SparkContext,
-    result: DataLoadResult[K, V],
-    carbonLoadModel: CarbonLoadModel,
-    storePath: String,
-    kettleHomePath: String,
-    columnar: Boolean,
-    loadCount: Integer,
-    tableCreationTime: Long,
-    schemaLastUpdatedTime: Long,
-    prev: DataLoadCoalescedRDD[Row]) extends RDD[(K, V)](prev) {
-
-  sc.setLocalProperty("spark.scheduler.pool", "DDL")
-
-  @DeveloperApi
-  override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val resultIter = new Iterator[(K, V)] {
-      val partitionID = "0"
-      val loadMetadataDetails = new LoadMetadataDetails()
-      val uniqueLoadStatusId = carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE +
-          theSplit.index
-      try {
-        loadMetadataDetails.setPartitionCount(partitionID)
-        carbonLoadModel.setPartitionId(partitionID)
-        carbonLoadModel.setSegmentId(String.valueOf(loadCount))
-        carbonLoadModel.setTaskNo(String.valueOf(theSplit.index))
-        val loader = new SparkPartitionLoader(carbonLoadModel, theSplit.index, storePath,
-          kettleHomePath, String.valueOf(loadCount), loadMetadataDetails)
-        loader.initialize
-        val rddIteratorKey = UUID.randomUUID().toString
-        try {
-          RddInputUtils.put(rddIteratorKey,
-              new PartitionIterator(
-                  firstParent[DataLoadPartitionWrap[Row]].iterator(theSplit, context),
-                  carbonLoadModel,
-                  context))
-          carbonLoadModel.setRddIteratorKey(rddIteratorKey)
-          loader.run()
-        } finally {
-          RddInputUtils.remove(rddIteratorKey)
-        }
-      } catch {
-        case e: Exception =>
-          logInfo("DataLoad failure")
-          LOGGER.error(e)
-          throw e
-      }
-
-      var finished = false
-
-      override def hasNext: Boolean = !finished
-
-      override def next(): (K, V) = {
-        finished = true
-        result.getKey(uniqueLoadStatusId, loadMetadataDetails)
-      }
-    }
-    resultIter
-  }
-
-  override protected def getPartitions: Array[Partition] = firstParent[Row].partitions
-}
-
-class PartitionIterator(partitionIter: Iterator[DataLoadPartitionWrap[Row]],
-    carbonLoadModel: CarbonLoadModel,
-    context: TaskContext) extends CarbonIterator[CarbonIterator[Array[String]]] {
-  val serializer = SparkEnv.get.closureSerializer.newInstance()
-  var serializeBuffer: ByteBuffer = null
-  def hasNext: Boolean = partitionIter.hasNext
-
-  def next: CarbonIterator[Array[String]] = {
-    val value = partitionIter.next
-    // The rdd (which come from Hive Table) don't support to read dataframe concurrently.
-    // So here will create different rdd instance for each thread.
-    val newInstance = {
-      if (serializeBuffer == null) {
-        serializeBuffer = serializer.serialize[RDD[Row]](value.rdd)
-      }
-      serializeBuffer.rewind()
-      serializer.deserialize[RDD[Row]](serializeBuffer)
-    }
-    new RddIterator(newInstance.iterator(value.partition, context),
-        carbonLoadModel,
-        context)
-  }
-  override def initialize: Unit = {
-    SparkUtil.setTaskContext(context)
-  }
-}
-/**
- * This class wrap Scala's Iterator to Java's Iterator.
- * It also convert all columns to string data to use csv data loading flow.
- *
- * @param rddIter
- * @param carbonLoadModel
- * @param context
- */
-class RddIterator(rddIter: Iterator[Row],
-                  carbonLoadModel: CarbonLoadModel,
-                  context: TaskContext) extends CarbonIterator[Array[String]] {
-
- val timeStampformatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
-    .CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-  val timeStampFormat = new SimpleDateFormat(timeStampformatString)
-  val dateFormatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
-    .CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-  val dateFormat = new SimpleDateFormat(dateFormatString)
-  val delimiterLevel1 = carbonLoadModel.getComplexDelimiterLevel1
-  val delimiterLevel2 = carbonLoadModel.getComplexDelimiterLevel2
-  val serializationNullFormat =
-      carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
-  def hasNext: Boolean = rddIter.hasNext
-
-  def next: Array[String] = {
-    val row = rddIter.next()
-    val columns = new Array[String](row.length)
-    for (i <- 0 until columns.length) {
-      columns(i) = CarbonScalaUtil.getString(row.get(i), serializationNullFormat,
-          delimiterLevel1, delimiterLevel2, timeStampFormat, dateFormat)
-    }
-    columns
-  }
-
-  override def initialize: Unit = {
-    SparkUtil.setTaskContext(context)
-  }
-
-}
-
-class RddIteratorForUpdate(rddIter: Iterator[Row],
-    carbonLoadModel: CarbonLoadModel) extends java.util.Iterator[Array[String]] {
-  val timeStampformatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
-    .CARBON_TIMESTAMP_FORMAT, CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
-  val timeStampFormat = new SimpleDateFormat(timeStampformatString)
-  val dateFormatString = CarbonProperties.getInstance().getProperty(CarbonCommonConstants
-    .CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT)
-  val dateFormat = new SimpleDateFormat(dateFormatString)
-  val delimiterLevel1 = carbonLoadModel.getComplexDelimiterLevel1
-  val delimiterLevel2 = carbonLoadModel.getComplexDelimiterLevel2
-  val serializationNullFormat =
-    carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
-
-  def hasNext: Boolean = rddIter.hasNext
-
-  def next: Array[String] = {
-    val row = rddIter.next()
-    val columns = new Array[String](row.length)
-    for (i <- 0 until row.length) {
-      // columns(i) = CarbonScalaUtil.getStringForUpdate(row(i), delimiterLevel1, delimiterLevel2)
-      columns(i) = CarbonScalaUtil.getString(row.get(i), serializationNullFormat,
-        delimiterLevel1, delimiterLevel2, timeStampFormat, dateFormat)
-      if (columns(i).length() > CarbonCommonConstants.DEFAULT_COLUMN_LENGTH) {
-        sys.error(s" Error processing input: Length of parsed input (${
-          CarbonCommonConstants
-            .DEFAULT_COLUMN_LENGTH
-        }) exceeds the maximum number of characters defined"
-        )
-      }
-    }
-    columns
-  }
-
-  def remove(): Unit = {
-  }
-}
-
-object CarbonDataLoadForUpdate {
-  def initialize(model: CarbonLoadModel,
-      splitIndex: Int): String = {
-    val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null)
-    if (null == carbonPropertiesFilePath) {
-      System.setProperty("carbon.properties.filepath",
-        System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties")
-    }
-    CarbonTimeStatisticsFactory.getLoadStatisticsInstance.initPartitonInfo(model.getPartitionId)
-    CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true")
-    CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1")
-    CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true")
-    CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true")
-    CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true")
-    CarbonProperties.getInstance().addProperty("high.cardinality.value", "100000")
-    CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false")
-    CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000")
-
-    // this property is used to determine whether temp location for carbon is inside
-    // container temp dir or is yarn application directory.
-    val carbonUseLocalDir = CarbonProperties.getInstance()
-      .getProperty("carbon.use.local.dir", "false")
-    var storeLocation = ""
-    if(carbonUseLocalDir.equalsIgnoreCase("true")) {
-      val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
-      if (null != storeLocations && storeLocations.nonEmpty) {
-        storeLocation = storeLocations(Random.nextInt(storeLocations.length))
-      }
-      if (storeLocation == null) {
-        storeLocation = System.getProperty("java.io.tmpdir")
-      }
-    }
-    else {
-      storeLocation = System.getProperty("java.io.tmpdir")
-    }
-    storeLocation = storeLocation + '/' + System.nanoTime() + '/' + splitIndex
-    storeLocation
-  }
-
-  def run(model: CarbonLoadModel,
-      index: Int,
-      hdfsStoreLocation: String,
-      kettleHomePath: String,
-      loadCount: String,
-      loadMetadataDetails: LoadMetadataDetails,
-      executorErrors: ExecutionErrors): Unit = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    try {
-      var storeLocation = ""
-      val carbonUseLocalDir = CarbonProperties.getInstance()
-        .getProperty("carbon.use.local.dir", "false")
-      if(carbonUseLocalDir.equalsIgnoreCase("true")) {
-        val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
-        if (null != storeLocations && storeLocations.nonEmpty) {
-          storeLocation = storeLocations(Random.nextInt(storeLocations.length))
-        }
-        if (storeLocation == null) {
-          storeLocation = System.getProperty("java.io.tmpdir")
-        }
-      }
-      else {
-        storeLocation = System.getProperty("java.io.tmpdir")
-      }
-      storeLocation = storeLocation + '/' + System.nanoTime() + '/' + index
-
-      CarbonLoaderUtil.executeGraph(model, storeLocation, hdfsStoreLocation,
-        kettleHomePath)
-      loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)
-    } catch {
-      case e: DataLoadingException => if (e.getErrorCode ==
-                                          DataProcessorConstants.BAD_REC_FOUND) {
-        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS)
-        LOGGER.info("Bad Record Found")
-      } else if (e.getErrorCode == DataProcessorConstants.BAD_REC_FAILURE_ERROR_CODE) {
-        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)
-        executorErrors.failureCauses = FailureCauses.BAD_RECORDS
-        executorErrors.errorMsg = e.getMessage
-      } else {
-        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)
-        throw e
-      }
-      case e: Exception =>
-        // this will be in case of any other exception where the executor has to rethrow and retry.
-        throw e
-    } finally {
-      // delete temp location data
-      try {
-        val isCompaction = false
-        CarbonLoaderUtil.deleteLocalDataLoadFolderLocation(model, isCompaction)
-      } catch {
-        case e: Exception =>
-          LOGGER.error("Failed to delete local data" + e)
-      }
-      if (!CarbonCommonConstants.STORE_LOADSTATUS_FAILURE.equals(
-        loadMetadataDetails.getLoadStatus)) {
-        CarbonTimeStatisticsFactory.getLoadStatisticsInstance.printStatisticsInfo(
-          model.getPartitionId)
-      }
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 d24d29a..ab0d603 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
@@ -220,9 +220,6 @@ class CarbonScanRDD(
           }
           if (!finished && !havePair) {
             finished = !reader.nextKeyValue
-            if (finished) {
-              reader.close()
-            }
             havePair = !finished
           }
           !finished

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
index 84c8ccf..3b38028 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/Compactor.scala
@@ -40,7 +40,6 @@ object Compactor {
     val storePath = compactionCallableModel.storePath
     val storeLocation = compactionCallableModel.storeLocation
     val carbonTable = compactionCallableModel.carbonTable
-    val kettleHomePath = compactionCallableModel.kettleHomePath
     val cubeCreationTime = compactionCallableModel.cubeCreationTime
     val loadsToMerge = compactionCallableModel.loadsToMerge
     val sc = compactionCallableModel.sqlContext
@@ -59,7 +58,6 @@ object Compactor {
       storePath,
       carbonTable.getMetaDataFilepath,
       mergedLoadName,
-      kettleHomePath,
       cubeCreationTime,
       databaseName,
       factTableName,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 3b3bac3..0ba99a8 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
@@ -150,7 +150,6 @@ object DataManagementFunc {
       compactionModel: CompactionModel,
       executor: ExecutorService,
       sqlContext: SQLContext,
-      kettleHomePath: String,
       storeLocation: String): Unit = {
     val sortedSegments: util.List[LoadMetadataDetails] = new util.ArrayList[LoadMetadataDetails](
       carbonLoadModel.getLoadMetadataDetails
@@ -181,7 +180,6 @@ object DataManagementFunc {
         storePath,
         sqlContext,
         compactionModel,
-        kettleHomePath,
         carbonLoadModel,
         storeLocation
       )
@@ -239,7 +237,6 @@ object DataManagementFunc {
       storePath: String,
       sqlContext: SQLContext,
       compactionModel: CompactionModel,
-      kettleHomePath: String,
       carbonLoadModel: CarbonLoadModel,
       storeLocation: String): Unit = {
 
@@ -252,7 +249,6 @@ object DataManagementFunc {
       carbonLoadModel,
       storeLocation,
       compactionModel.carbonTable,
-      kettleHomePath,
       compactionModel.tableCreationTime,
       loadsToMerge,
       sqlContext,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 cb2bd3e..50894d4 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
@@ -25,12 +25,13 @@ import java.util.{Date, UUID}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable
+import scala.util.Random
 import scala.util.control.NonFatal
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
-import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext}
+import org.apache.spark.{Partition, SerializableWritable, SparkContext, SparkEnv, TaskContext}
 import org.apache.spark.rdd.{DataLoadCoalescedRDD, DataLoadPartitionWrap, RDD}
 import org.apache.spark.sql.Row
 import org.apache.spark.util.SparkUtil
@@ -41,13 +42,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}
+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.csvreaderstep.BlockDetails
 import org.apache.carbondata.processing.model.CarbonLoadModel
 import org.apache.carbondata.processing.newflow.DataLoadExecutor
 import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException
 import org.apache.carbondata.spark.DataLoadResult
+import org.apache.carbondata.spark.load.CarbonLoaderUtil
 import org.apache.carbondata.spark.splits.TableSplit
 import org.apache.carbondata.spark.util.{CarbonQueryUtil, CarbonScalaUtil, CommonUtil}
 
@@ -85,6 +87,81 @@ class SerializableConfiguration(@transient var value: Configuration) extends Ser
 }
 
 /**
+ * This partition class use to split by Host
+ *
+ */
+class CarbonNodePartition(rddId: Int, val idx: Int, host: String,
+    val blocksDetails: Array[BlockDetails])
+  extends Partition {
+
+  override val index: Int = idx
+  val serializableHadoopSplit = host
+  val nodeBlocksDetail = blocksDetails
+
+  override def hashCode(): Int = 41 * (41 + rddId) + idx
+}
+
+/**
+ * This partition class use to split by TableSplit
+ *
+ */
+class CarbonTableSplitPartition(rddId: Int, val idx: Int, @transient val tableSplit: TableSplit,
+    val blocksDetails: Array[BlockDetails])
+  extends Partition {
+
+  override val index: Int = idx
+  val serializableHadoopSplit = new SerializableWritable[TableSplit](tableSplit)
+  val partitionBlocksDetail = blocksDetails
+
+  override def hashCode(): Int = 41 * (41 + rddId) + idx
+}
+
+class SparkPartitionLoader(model: CarbonLoadModel,
+    splitIndex: Int,
+    storePath: String,
+    loadCount: String,
+    loadMetadataDetails: LoadMetadataDetails) {
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  var storeLocation: String = ""
+
+  def initialize(): Unit = {
+    val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null)
+    if (null == carbonPropertiesFilePath) {
+      System.setProperty("carbon.properties.filepath",
+        System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties")
+    }
+    CarbonTimeStatisticsFactory.getLoadStatisticsInstance.initPartitonInfo(model.getPartitionId)
+    CarbonProperties.getInstance().addProperty("carbon.is.columnar.storage", "true")
+    CarbonProperties.getInstance().addProperty("carbon.dimension.split.value.in.columnar", "1")
+    CarbonProperties.getInstance().addProperty("carbon.is.fullyfilled.bits", "true")
+    CarbonProperties.getInstance().addProperty("is.int.based.indexer", "true")
+    CarbonProperties.getInstance().addProperty("aggregate.columnar.keyblock", "true")
+    CarbonProperties.getInstance().addProperty("high.cardinality.value", "100000")
+    CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false")
+    CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000")
+
+    // this property is used to determine whether temp location for carbon is inside
+    // container temp dir or is yarn application directory.
+    val carbonUseLocalDir = CarbonProperties.getInstance()
+      .getProperty("carbon.use.local.dir", "false")
+    if (carbonUseLocalDir.equalsIgnoreCase("true")) {
+      val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
+      if (null != storeLocations && storeLocations.nonEmpty) {
+        storeLocation = storeLocations(Random.nextInt(storeLocations.length))
+      }
+      if (storeLocation == null) {
+        storeLocation = System.getProperty("java.io.tmpdir")
+      }
+    } else {
+      storeLocation = System.getProperty("java.io.tmpdir")
+    }
+    storeLocation = storeLocation + '/' + System.nanoTime() + '/' + splitIndex
+  }
+
+}
+
+/**
  * It loads the data to carbon using @AbstractDataLoadProcessorStep
  */
 class NewCarbonDataLoadRDD[K, V](
@@ -157,7 +234,6 @@ class NewCarbonDataLoadRDD[K, V](
         val loader = new SparkPartitionLoader(model,
           theSplit.index,
           null,
-          null,
           String.valueOf(loadCount),
           loadMetadataDetails)
         // Intialize to set carbon properties
@@ -304,8 +380,7 @@ class NewCarbonDataLoadRDD[K, V](
 
 /**
  *  It loads the data to carbon from spark DataFrame using
- *  @see org.apache.carbondata.processing.newflow.DataLoadExecutor without
- *  kettle requirement
+ *  @see org.apache.carbondata.processing.newflow.DataLoadExecutor
  */
 class NewDataFrameLoaderRDD[K, V](
                                    sc: SparkContext,
@@ -355,7 +430,6 @@ class NewDataFrameLoaderRDD[K, V](
         val loader = new SparkPartitionLoader(model,
           theSplit.index,
           null,
-          null,
           String.valueOf(loadCount),
           loadMetadataDetails)
         // Intialize to set carbon properties

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 bb661b1..a36fb63 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
@@ -31,11 +31,11 @@ import org.apache.carbondata.processing.newflow.DataLoadExecutor
 import org.apache.carbondata.processing.newflow.exception.BadRecordFoundException
 
 /**
- * Data load in case of update command with out kettle.
+ * Data load in case of update command .
  */
 object UpdateDataLoad {
 
-  def DataLoadNoKettleForUpdate(segId: String,
+  def DataLoadForUpdate(segId: String,
       index: Int,
       iter: Iterator[Row],
       carbonLoadModel: CarbonLoadModel,
@@ -50,7 +50,6 @@ object UpdateDataLoad {
       val loader = new SparkPartitionLoader(carbonLoadModel,
         index,
         null,
-        null,
         segId,
         loadMetadataDetails)
       // Intialize to set carbon properties

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index 3a0e395..c95b7ea 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -93,58 +93,6 @@ object CarbonScalaUtil {
     }
   }
 
-  def getKettleHome(sqlContext: SQLContext): String = {
-    var kettleHomePath = sqlContext.getConf("carbon.kettle.home", null)
-    if (null == kettleHomePath) {
-      kettleHomePath = CarbonProperties.getInstance.getProperty("carbon.kettle.home")
-    }
-    if (null == kettleHomePath) {
-      val carbonHome = System.getenv("CARBON_HOME")
-      if (null != carbonHome) {
-        kettleHomePath = carbonHome + "/processing/carbonplugins"
-      }
-    }
-    if (kettleHomePath != null) {
-      val sparkMaster = sqlContext.sparkContext.getConf.get("spark.master").toLowerCase()
-      // get spark master, if local, need to correct the kettle home
-      // e.g: --master local, the executor running in local machine
-      if (sparkMaster.startsWith("local")) {
-        val kettleHomeFileType = FileFactory.getFileType(kettleHomePath)
-        val kettleHomeFile = FileFactory.getCarbonFile(kettleHomePath, kettleHomeFileType)
-        // check if carbon.kettle.home path is exists
-        if (!kettleHomeFile.exists()) {
-          // get the path of this class
-          // e.g: file:/srv/bigdata/install/spark/sparkJdbc/carbonlib/carbon-
-          // xxx.jar!/org/carbondata/spark/rdd/
-          var jarFilePath = this.getClass.getResource("").getPath
-          val endIndex = jarFilePath.indexOf(".jar!") + 4
-          // get the jar file path
-          // e.g: file:/srv/bigdata/install/spark/sparkJdbc/carbonlib/carbon-*.jar
-          jarFilePath = jarFilePath.substring(0, endIndex)
-          val jarFileType = FileFactory.getFileType(jarFilePath)
-          val jarFile = FileFactory.getCarbonFile(jarFilePath, jarFileType)
-          // get the parent folder of the jar file
-          // e.g:file:/srv/bigdata/install/spark/sparkJdbc/carbonlib
-          val carbonLibPath = jarFile.getParentFile.getPath
-          // find the kettle home under the previous folder
-          // e.g:file:/srv/bigdata/install/spark/sparkJdbc/carbonlib/cabonplugins
-          kettleHomePath = carbonLibPath + File.separator + CarbonCommonConstants.KETTLE_HOME_NAME
-          val logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-          logger.error(s"carbon.kettle.home path is not exists, reset it as $kettleHomePath")
-          val newKettleHomeFileType = FileFactory.getFileType(kettleHomePath)
-          val newKettleHomeFile = FileFactory.getCarbonFile(kettleHomePath, newKettleHomeFileType)
-          // check if the found kettle home exists
-          if (!newKettleHomeFile.exists()) {
-            sys.error("Kettle home not found. Failed to reset carbon.kettle.home")
-          }
-        }
-      }
-    } else {
-      sys.error("carbon.kettle.home is not set")
-    }
-    kettleHomePath
-  }
-
   def getString(value: Any,
       serializationNullFormat: String,
       delimiterLevel1: String,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 9c5ce83..8120942 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
@@ -751,7 +751,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
     val supportedOptions = Seq("DELIMITER", "QUOTECHAR", "FILEHEADER", "ESCAPECHAR", "MULTILINE",
       "COMPLEX_DELIMITER_LEVEL_1", "COMPLEX_DELIMITER_LEVEL_2", "COLUMNDICT",
       "SERIALIZATION_NULL_FORMAT", "BAD_RECORDS_LOGGER_ENABLE", "BAD_RECORDS_ACTION",
-      "ALL_DICTIONARY_PATH", "MAXCOLUMNS", "COMMENTCHAR", "USE_KETTLE", "DATEFORMAT",
+      "ALL_DICTIONARY_PATH", "MAXCOLUMNS", "COMMENTCHAR", "DATEFORMAT",
       "SINGLE_PASS", "IS_EMPTY_DATA_BAD_RECORD"
     )
     var isSupported = true

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index c858d0a..3346743 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -86,7 +86,6 @@ case class CarbonMergerMapping(storeLocation: String,
     hdfsStoreLocation: String,
     metadataFilePath: String,
     var mergedLoadName: String,
-    kettleHomePath: String,
     tableCreationTime: Long,
     databaseName: String,
     factTableName: String,
@@ -121,7 +120,6 @@ case class CompactionCallableModel(storePath: String,
     carbonLoadModel: CarbonLoadModel,
     storeLocation: String,
     carbonTable: CarbonTable,
-    kettleHomePath: String,
     cubeCreationTime: Long,
     loadsToMerge: util.List[LoadMetadataDetails],
     sqlContext: SQLContext,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
index f8e2bc1..a5fef5e 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/test/TestQueryExecutor.scala
@@ -48,7 +48,6 @@ object TestQueryExecutor {
   val storeLocation = s"$integrationPath/spark-common/target/store"
   val warehouse = s"$integrationPath/spark-common/target/warehouse"
   val metastoredb = s"$integrationPath/spark-common/target"
-  val kettleHome = s"$projectPath/processing/carbonplugins"
   val timestampFormat = "dd-MM-yyyy"
 
   val INSTANCE = lookupQueryExecutor.newInstance().asInstanceOf[TestQueryExecutorRegister]

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 7c6665c..643002d 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.csvreaderstep.BlockDetails
+import org.apache.carbondata.processing.csvload.BlockDetails
 
 /*
  * this object use to handle file splits

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark/pom.xml b/integration/spark/pom.xml
index 1a288a8..6083fba 100644
--- a/integration/spark/pom.xml
+++ b/integration/spark/pom.xml
@@ -175,7 +175,6 @@
           </environmentVariables>
           <systemProperties>
             <java.awt.headless>true</java.awt.headless>
-            <use_kettle>${use.kettle}</use_kettle>
           </systemProperties>
         </configuration>
         <executions>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala
index ebeeed2..0e2e4dd 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonDataFrameWriter.scala
@@ -186,19 +186,11 @@ class CarbonDataFrameWriter(val dataFrame: DataFrame) {
   }
 
   private def makeLoadString(csvFolder: String, options: CarbonOption): String = {
-    if (options.useKettle) {
-      s"""
-          LOAD DATA INPATH '$csvFolder'
-          INTO TABLE ${options.dbName}.${options.tableName}
-          OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}', 'USE_KETTLE' = 'true')
-      """
-    } else {
-      s"""
+    s"""
           LOAD DATA INPATH '$csvFolder'
           INTO TABLE ${options.dbName}.${options.tableName}
-          OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}', 'USE_KETTLE' = 'false')
+          OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}')
       """
-    }
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 dbd2a21..60742ac 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
@@ -48,7 +48,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.processing.csvreaderstep.{BlockDetails, RddInpututilsForUpdate}
+import org.apache.carbondata.processing.csvload.BlockDetails
 import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.carbondata.processing.model.CarbonLoadModel
 import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
@@ -70,7 +70,6 @@ object CarbonDataRDDFactory {
       alterTableModel: AlterTableModel,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
-      kettleHomePath: String,
       storeLocation: String): Unit = {
     var compactionSize: Long = 0
     var compactionType: CompactionType = CompactionType.MINOR_COMPACTION
@@ -130,7 +129,6 @@ object CarbonDataRDDFactory {
       handleCompactionForSystemLocking(sqlContext,
         carbonLoadModel,
         storePath,
-        kettleHomePath,
         storeLocation,
         compactionType,
         carbonTable,
@@ -150,7 +148,6 @@ object CarbonDataRDDFactory {
           startCompactionThreads(sqlContext,
             carbonLoadModel,
             storePath,
-            kettleHomePath,
             storeLocation,
             compactionModel,
             lock
@@ -173,7 +170,6 @@ object CarbonDataRDDFactory {
   def handleCompactionForSystemLocking(sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
-      kettleHomePath: String,
       storeLocation: String,
       compactionType: CompactionType,
       carbonTable: CarbonTable,
@@ -189,7 +185,6 @@ object CarbonDataRDDFactory {
         startCompactionThreads(sqlContext,
           carbonLoadModel,
           storePath,
-          kettleHomePath,
           storeLocation,
           compactionModel,
           lock
@@ -226,7 +221,6 @@ object CarbonDataRDDFactory {
   def startCompactionThreads(sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
-      kettleHomePath: String,
       storeLocation: String,
       compactionModel: CompactionModel,
       compactionLock: ICarbonLock): Unit = {
@@ -254,7 +248,7 @@ object CarbonDataRDDFactory {
             DataManagementFunc.executeCompaction(carbonLoadModel: CarbonLoadModel,
               storePath: String,
               compactionModel: CompactionModel,
-              executor, sqlContext, kettleHomePath, storeLocation
+              executor, sqlContext, storeLocation
             )
             triggeredCompactionStatus = true
           } catch {
@@ -303,7 +297,7 @@ object CarbonDataRDDFactory {
                 DataManagementFunc.executeCompaction(newCarbonLoadModel,
                   newCarbonLoadModel.getStorePath,
                   newcompactionModel,
-                  executor, sqlContext, kettleHomePath, storeLocation
+                  executor, sqlContext, storeLocation
                 )
               } catch {
                 case e: Exception =>
@@ -351,10 +345,8 @@ object CarbonDataRDDFactory {
   def loadCarbonData(sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
-      kettleHomePath: String,
       columnar: Boolean,
       partitionStatus: String = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS,
-      useKettle: Boolean,
       result: Future[DictionaryServer],
       dataFrame: Option[DataFrame] = None,
       updateModel: Option[UpdateTableModel] = None): Unit = {
@@ -396,7 +388,6 @@ object CarbonDataRDDFactory {
           handleCompactionForSystemLocking(sqlContext,
             carbonLoadModel,
             storePath,
-            kettleHomePath,
             storeLocation,
             CompactionType.MINOR_COMPACTION,
             carbonTable,
@@ -414,7 +405,6 @@ object CarbonDataRDDFactory {
               startCompactionThreads(sqlContext,
                 carbonLoadModel,
                 storePath,
-                kettleHomePath,
                 storeLocation,
                 compactionModel,
                 lock
@@ -444,10 +434,6 @@ object CarbonDataRDDFactory {
     try {
       LOGGER.audit(s"Data load request has been received for table" +
           s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-      if (!useKettle) {
-        LOGGER.audit("Data is loading with New Data Flow for table " +
-            s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-      }
       // Check if any load need to be deleted before loading new data
       DataManagementFunc.deleteLoadsAndUpdateMetadata(carbonLoadModel.getDatabaseName,
         carbonLoadModel.getTableName, storePath, isForceDeletion = false)
@@ -555,8 +541,7 @@ object CarbonDataRDDFactory {
            * 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info
            * 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block,
            *   for locally writing carbondata files(one file one block) in nodes
-           * 4)use kettle: use DataFileLoaderRDD to load data and write to carbondata files
-           *   non kettle: use NewCarbonDataLoadRDD to load data and write to carbondata files
+           *   use NewCarbonDataLoadRDD to load data and write to carbondata files
            */
           val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
           // FileUtils will skip file which is no csv, and return all file path which split by ','
@@ -621,27 +606,12 @@ object CarbonDataRDDFactory {
           ).toArray
         }
 
-        if (useKettle) {
-          status = new DataFileLoaderRDD(sqlContext.sparkContext,
-            new DataLoadResultImpl(),
-            carbonLoadModel,
-            storePath,
-            kettleHomePath,
-            columnar,
-            currentLoadCount,
-            tableCreationTime,
-            schemaLastUpdatedTime,
-            blocksGroupBy,
-            isTableSplitPartition
-          ).collect()
-        } else {
-          status = new NewCarbonDataLoadRDD(sqlContext.sparkContext,
-            new DataLoadResultImpl(),
-            carbonLoadModel,
-            currentLoadCount,
-            blocksGroupBy,
-            isTableSplitPartition).collect()
-        }
+        status = new NewCarbonDataLoadRDD(sqlContext.sparkContext,
+          new DataLoadResultImpl(),
+          carbonLoadModel,
+          currentLoadCount,
+          blocksGroupBy,
+          isTableSplitPartition).collect()
       }
 
       def loadDataFrame(): Unit = {
@@ -653,31 +623,17 @@ object CarbonDataRDDFactory {
           val nodes = DistributionUtil.ensureExecutorsByNumberAndGetNodeList(nodeNumOfData,
             sqlContext.sparkContext)
           val newRdd = new DataLoadCoalescedRDD[Row](rdd, nodes.toArray.distinct)
-          if (useKettle) {
-            status = new DataFrameLoaderRDD(sqlContext.sparkContext,
-              new DataLoadResultImpl(),
-              carbonLoadModel,
-              storePath,
-              kettleHomePath,
-              columnar,
-              currentLoadCount,
-              tableCreationTime,
-              schemaLastUpdatedTime,
-              newRdd).collect()
-          } else {
+          var numPartitions = DistributionUtil.getNodeList(sqlContext.sparkContext).length
+          numPartitions = Math.max(1, Math.min(numPartitions, rdd.partitions.length))
+          val coalesceRdd = rdd.coalesce(numPartitions, shuffle = false)
 
-            var numPartitions = DistributionUtil.getNodeList(sqlContext.sparkContext).length
-            numPartitions = Math.max(1, Math.min(numPartitions, rdd.partitions.length))
-            val coalesceRdd = rdd.coalesce(numPartitions, shuffle = false)
-
-            status = new NewDataFrameLoaderRDD(sqlContext.sparkContext,
-              new DataLoadResultImpl(),
-              carbonLoadModel,
-              currentLoadCount,
-              tableCreationTime,
-              schemaLastUpdatedTime,
-              newRdd).collect()
-          }
+          status = new NewDataFrameLoaderRDD(sqlContext.sparkContext,
+            new DataLoadResultImpl(),
+            carbonLoadModel,
+            currentLoadCount,
+            tableCreationTime,
+            schemaLastUpdatedTime,
+            newRdd).collect()
         } catch {
           case ex: Exception =>
             LOGGER.error(ex, "load data frame failed")
@@ -726,29 +682,11 @@ object CarbonDataRDDFactory {
               loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)
               val rddIteratorKey = CarbonCommonConstants.RDDUTIL_UPDATE_KEY +
                                    UUID.randomUUID().toString
-              if (useKettle) {
-                try {
-                  RddInpututilsForUpdate.put(rddIteratorKey,
-                    new RddIteratorForUpdate(iter, carbonLoadModel))
-                  carbonLoadModel.setRddIteratorKey(rddIteratorKey)
-                  CarbonDataLoadForUpdate.run(carbonLoadModel,
-                    index,
-                    storePath,
-                    kettleHomePath,
-                    segId,
-                    loadMetadataDetails,
-                    executionErrors)
-                } finally {
-                  RddInpututilsForUpdate.remove(rddIteratorKey)
-                }
-              } else {
-                UpdateDataLoad.DataLoadNoKettleForUpdate(segId,
-                  index,
-                  iter,
-                  carbonLoadModel,
-                  loadMetadataDetails)
-
-              }
+              UpdateDataLoad.DataLoadForUpdate(segId,
+                index,
+                iter,
+                carbonLoadModel,
+                loadMetadataDetails)
             } catch {
               case e: Exception =>
                 LOGGER.info("DataLoad failure")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 6f14feb..5a22e9c 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
@@ -110,8 +110,6 @@ private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) e
     carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
     carbonLoadModel.setStorePath(relation.tableMeta.storePath)
 
-    val kettleHomePath = CarbonScalaUtil.getKettleHome(sqlContext)
-
     var storeLocation = CarbonProperties.getInstance
       .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
         System.getProperty("java.io.tmpdir")
@@ -122,7 +120,6 @@ private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) e
           alterTableModel,
           carbonLoadModel,
           relation.tableMeta.storePath,
-          kettleHomePath,
           storeLocation
         )
     } catch {
@@ -387,28 +384,6 @@ case class LoadTable(
 
       val columnar = sqlContext.getConf("carbon.is.columnar.storage", "true").toBoolean
 
-      // TODO It will be removed after kettle is removed.
-      val useKettle = options.get("use_kettle") match {
-        case Some(value) => value.toBoolean
-        case _ =>
-          var useKettleLocal = System.getProperty("use_kettle")
-          if (useKettleLocal == null && sqlContext.sparkContext.getConf.contains("use_kettle")) {
-            useKettleLocal = sqlContext.sparkContext.getConf.get("use_kettle")
-          }
-          if (useKettleLocal == null) {
-            useKettleLocal = CarbonProperties.getInstance().
-              getProperty(CarbonCommonConstants.USE_KETTLE,
-                CarbonCommonConstants.USE_KETTLE_DEFAULT)
-          }
-          try {
-            useKettleLocal.toBoolean
-          } catch {
-            case e: Exception => CarbonCommonConstants.USE_KETTLE_DEFAULT.toBoolean
-          }
-      }
-
-      val kettleHomePath = if (useKettle) CarbonScalaUtil.getKettleHome(sqlContext) else ""
-
       val delimiter = options.getOrElse("delimiter", ",")
       val quoteChar = options.getOrElse("quotechar", "\"")
       val fileHeader = options.getOrElse("fileheader", "")
@@ -452,14 +427,14 @@ case class LoadTable(
       carbonLoadModel
         .setIsEmptyDataBadRecord(
           DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + isEmptyDataBadRecord)
-      // when single_pass=true, and use_kettle=false, and not use all dict
+      // when single_pass=true, and not use all dict
       val useOnePass = options.getOrElse("single_pass", "false").trim.toLowerCase match {
         case "true" =>
-          if (!useKettle && StringUtils.isEmpty(allDictionaryPath)) {
+          if (StringUtils.isEmpty(allDictionaryPath)) {
             true
           } else {
             LOGGER.error("Can't use single_pass, because SINGLE_PASS and ALL_DICTIONARY_PATH" +
-              "can not be used together, and USE_KETTLE must be set as false")
+              "can not be used together")
             false
           }
         case "false" =>
@@ -540,10 +515,8 @@ case class LoadTable(
           CarbonDataRDDFactory.loadCarbonData(sqlContext,
             carbonLoadModel,
             relation.tableMeta.storePath,
-            kettleHomePath,
             columnar,
             partitionStatus,
-            useKettle,
             result,
             dataFrame,
             updateModel)
@@ -587,10 +560,8 @@ case class LoadTable(
           CarbonDataRDDFactory.loadCarbonData(sqlContext,
             carbonLoadModel,
             relation.tableMeta.storePath,
-            kettleHomePath,
             columnar,
             partitionStatus,
-            useKettle,
             result,
             loadDataFrame,
             updateModel)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala b/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala
index d9c6fbb..591cdf4 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala
@@ -38,7 +38,6 @@ object SparkTestQueryExecutor {
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
   LOGGER.info("use TestQueryExecutorImplV1")
   CarbonProperties.getInstance()
-    .addProperty("carbon.kettle.home", TestQueryExecutor.kettleHome)
     .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, TestQueryExecutor.timestampFormat)
     .addProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
       System.getProperty("java.io.tmpdir"))
@@ -47,8 +46,7 @@ object SparkTestQueryExecutor {
   val sc = new SparkContext(new SparkConf()
     .setAppName("CarbonSpark")
     .setMaster("local[2]")
-    .set("spark.sql.shuffle.partitions", "20")
-    .set("use_kettle_default", "true"))
+    .set("spark.sql.shuffle.partitions", "20"))
   sc.setLogLevel("ERROR")
 
   val cc = new CarbonContext(sc, TestQueryExecutor.storeLocation, TestQueryExecutor.metastoredb)



[10/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala
index 885a0f0..851f7e9 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/badrecordloger/BadRecordLoggerSharedDictionaryTest.scala
@@ -69,18 +69,6 @@ class BadRecordLoggerSharedDictionaryTest extends QueryTest with BeforeAndAfterA
       sql(
         s"""LOAD DATA INPATH '$csvFilePath' INTO TABLE testdrive OPTIONS('DELIMITER'=',',
             |'QUOTECHAR'= '"', 'BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FAIL',
-            |'FILEHEADER'= 'ID,CUST_ID,cust_name', 'USE_KETTLE' = 'TRUE')""".stripMargin)
-    } catch {
-      case e: Throwable =>
-        assert(e.getMessage.contains("Data load failed due to bad record"))
-    }
-  }
-
-  test("dataload with no kettle") {
-    try {
-      sql(
-        s"""LOAD DATA INPATH '$csvFilePath' INTO TABLE testdrive OPTIONS('DELIMITER'=',',
-            |'QUOTECHAR'= '"', 'BAD_RECORDS_LOGGER_ENABLE'='TRUE', 'BAD_RECORDS_ACTION'='FAIL',
             |'FILEHEADER'= 'ID,CUST_ID,cust_name')""".stripMargin)
     } catch {
       case e: Throwable =>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala
index 8a9d8af..1d456d3 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithSinglePass.scala
@@ -41,7 +41,7 @@ class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll {
     sql(
       s"""
         |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_two_pass
-        |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='false')
+        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='false')
       """.stripMargin)
 
     sql(
@@ -54,7 +54,7 @@ class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll {
     sql(
       s"""
         |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_one_pass
-        |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='true')
+        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true')
       """.stripMargin)
   }
 
@@ -75,7 +75,7 @@ class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll {
     sql(
       s"""
         |LOAD DATA local inpath '$resourcesPath/source.csv' INTO TABLE table_one_pass_2
-        |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='true', 'COLUMNDICT'=
+        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true', 'COLUMNDICT'=
         |'country:$resourcesPath/columndictionary/country.csv, name:$resourcesPath/columndictionary/name.csv')
       """.stripMargin)
 
@@ -89,12 +89,12 @@ class TestLoadDataWithSinglePass extends QueryTest with BeforeAndAfterAll {
     sql(
       s"""
         |LOAD DATA local inpath '$resourcesPath/dataIncrement.csv' INTO TABLE table_two_pass
-        |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='false')
+        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='false')
       """.stripMargin)
     sql(
       s"""
         |LOAD DATA local inpath '$resourcesPath/dataIncrement.csv' INTO TABLE table_one_pass
-        |OPTIONS('DELIMITER'= ',', 'USE_KETTLE'='false', 'SINGLE_PASS'='true')
+        |OPTIONS('DELIMITER'= ',', 'SINGLE_PASS'='true')
       """.stripMargin)
 
     checkAnswer(

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark2/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark2/pom.xml b/integration/spark2/pom.xml
index 44bce5e..f827792 100644
--- a/integration/spark2/pom.xml
+++ b/integration/spark2/pom.xml
@@ -156,7 +156,6 @@
           </environmentVariables>
           <systemProperties>
             <java.awt.headless>true</java.awt.headless>
-            <use_kettle>${use.kettle}</use_kettle>
           </systemProperties>
         </configuration>
         <executions>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 2fc918d..7cb5ed4 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
@@ -48,7 +48,7 @@ import org.apache.carbondata.core.mutate.CarbonUpdateUtil
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails
 import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonStorePath
-import org.apache.carbondata.processing.csvreaderstep.{BlockDetails, RddInpututilsForUpdate}
+import org.apache.carbondata.processing.csvload.BlockDetails
 import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.carbondata.processing.model.CarbonLoadModel
 import org.apache.carbondata.processing.newflow.exception.CarbonDataLoadingException
@@ -70,7 +70,6 @@ object CarbonDataRDDFactory {
       alterTableModel: AlterTableModel,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
-      kettleHomePath: String,
       storeLocation: String): Unit = {
     var compactionSize: Long = 0
     var compactionType: CompactionType = CompactionType.MINOR_COMPACTION
@@ -136,7 +135,6 @@ object CarbonDataRDDFactory {
       handleCompactionForSystemLocking(sqlContext,
         carbonLoadModel,
         storePath,
-        kettleHomePath,
         storeLocation,
         compactionType,
         carbonTable,
@@ -156,7 +154,6 @@ object CarbonDataRDDFactory {
           startCompactionThreads(sqlContext,
             carbonLoadModel,
             storePath,
-            kettleHomePath,
             storeLocation,
             compactionModel,
             lock
@@ -179,7 +176,6 @@ object CarbonDataRDDFactory {
   def handleCompactionForSystemLocking(sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
-      kettleHomePath: String,
       storeLocation: String,
       compactionType: CompactionType,
       carbonTable: CarbonTable,
@@ -195,7 +191,6 @@ object CarbonDataRDDFactory {
         startCompactionThreads(sqlContext,
           carbonLoadModel,
           storePath,
-          kettleHomePath,
           storeLocation,
           compactionModel,
           lock
@@ -232,7 +227,6 @@ object CarbonDataRDDFactory {
   def startCompactionThreads(sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
-      kettleHomePath: String,
       storeLocation: String,
       compactionModel: CompactionModel,
       compactionLock: ICarbonLock): Unit = {
@@ -264,7 +258,7 @@ object CarbonDataRDDFactory {
             DataManagementFunc.executeCompaction(carbonLoadModel: CarbonLoadModel,
               storePath: String,
               compactionModel: CompactionModel,
-              executor, sqlContext, kettleHomePath, storeLocation
+              executor, sqlContext, storeLocation
             )
             triggeredCompactionStatus = true
           } catch {
@@ -313,7 +307,7 @@ object CarbonDataRDDFactory {
                 DataManagementFunc.executeCompaction(newCarbonLoadModel,
                   newCarbonLoadModel.getStorePath,
                   newcompactionModel,
-                  executor, sqlContext, kettleHomePath, storeLocation
+                  executor, sqlContext, storeLocation
                 )
               } catch {
                 case e: Exception =>
@@ -361,10 +355,8 @@ object CarbonDataRDDFactory {
   def loadCarbonData(sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,
       storePath: String,
-      kettleHomePath: String,
       columnar: Boolean,
       partitionStatus: String = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS,
-      useKettle: Boolean,
       result: Future[DictionaryServer],
       dataFrame: Option[DataFrame] = None,
       updateModel: Option[UpdateTableModel] = None): Unit = {
@@ -406,7 +398,6 @@ object CarbonDataRDDFactory {
           handleCompactionForSystemLocking(sqlContext,
             carbonLoadModel,
             storePath,
-            kettleHomePath,
             storeLocation,
             CompactionType.MINOR_COMPACTION,
             carbonTable,
@@ -424,7 +415,6 @@ object CarbonDataRDDFactory {
               startCompactionThreads(sqlContext,
                 carbonLoadModel,
                 storePath,
-                kettleHomePath,
                 storeLocation,
                 compactionModel,
                 lock
@@ -454,10 +444,6 @@ object CarbonDataRDDFactory {
     try {
       LOGGER.audit(s"Data load request has been received for table" +
           s" ${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-      if (!useKettle) {
-        LOGGER.audit("Data is loading with New Data Flow for table " +
-            s"${ carbonLoadModel.getDatabaseName }.${ carbonLoadModel.getTableName }")
-      }
       // Check if any load need to be deleted before loading new data
       DataManagementFunc.deleteLoadsAndUpdateMetadata(carbonLoadModel.getDatabaseName,
         carbonLoadModel.getTableName, storePath, isForceDeletion = false)
@@ -565,8 +551,7 @@ object CarbonDataRDDFactory {
            * 2)use org.apache.hadoop.mapreduce.lib.input.TextInputFormat to get splits,size info
            * 3)use CarbonLoaderUtil.nodeBlockMapping to get mapping info of node and block,
            *   for locally writing carbondata files(one file one block) in nodes
-           * 4)use kettle: use DataFileLoaderRDD to load data and write to carbondata files
-           *   non kettle: use NewCarbonDataLoadRDD to load data and write to carbondata files
+           * 4)use NewCarbonDataLoadRDD to load data and write to carbondata files
            */
           val hadoopConfiguration = new Configuration(sqlContext.sparkContext.hadoopConfiguration)
           // FileUtils will skip file which is no csv, and return all file path which split by ','
@@ -631,27 +616,12 @@ object CarbonDataRDDFactory {
           ).toArray
         }
 
-        if (useKettle) {
-          status = new DataFileLoaderRDD(sqlContext.sparkContext,
-            new DataLoadResultImpl(),
-            carbonLoadModel,
-            storePath,
-            kettleHomePath,
-            columnar,
-            currentLoadCount,
-            tableCreationTime,
-            schemaLastUpdatedTime,
-            blocksGroupBy,
-            isTableSplitPartition
-          ).collect()
-        } else {
-          status = new NewCarbonDataLoadRDD(sqlContext.sparkContext,
-            new DataLoadResultImpl(),
-            carbonLoadModel,
-            currentLoadCount,
-            blocksGroupBy,
-            isTableSplitPartition).collect()
-        }
+        status = new NewCarbonDataLoadRDD(sqlContext.sparkContext,
+          new DataLoadResultImpl(),
+          carbonLoadModel,
+          currentLoadCount,
+          blocksGroupBy,
+          isTableSplitPartition).collect()
       }
 
       def loadDataFrame(): Unit = {
@@ -665,26 +635,13 @@ object CarbonDataRDDFactory {
             sqlContext.sparkContext)
           val newRdd = new DataLoadCoalescedRDD[Row](rdd, nodes.toArray.distinct)
 
-          if (useKettle) {
-            status = new DataFrameLoaderRDD(sqlContext.sparkContext,
-              new DataLoadResultImpl(),
-              carbonLoadModel,
-              storePath,
-              kettleHomePath,
-              columnar,
-              currentLoadCount,
-              tableCreationTime,
-              schemaLastUpdatedTime,
-              newRdd).collect()
-          } else {
-            status = new NewDataFrameLoaderRDD(sqlContext.sparkContext,
-              new DataLoadResultImpl(),
-              carbonLoadModel,
-              currentLoadCount,
-              tableCreationTime,
-              schemaLastUpdatedTime,
-              newRdd).collect()
-          }
+          status = new NewDataFrameLoaderRDD(sqlContext.sparkContext,
+            new DataLoadResultImpl(),
+            carbonLoadModel,
+            currentLoadCount,
+            tableCreationTime,
+            schemaLastUpdatedTime,
+            newRdd).collect()
 
         } catch {
           case ex: Exception =>
@@ -693,103 +650,6 @@ object CarbonDataRDDFactory {
         }
       }
 
-      def loadDataFrameForUpdate(): Unit = {
-        def triggerDataLoadForSegment(key: String,
-            iter: Iterator[Row]): Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] = {
-          val rddResult = new updateResultImpl()
-          val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-          val resultIter = new Iterator[(String, (LoadMetadataDetails, ExecutionErrors))] {
-            var partitionID = "0"
-            val loadMetadataDetails = new LoadMetadataDetails
-            val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
-            var uniqueLoadStatusId = ""
-            try {
-              val segId = key
-              val taskNo = CarbonUpdateUtil
-                .getLatestTaskIdForSegment(segId,
-                  CarbonStorePath.getCarbonTablePath(carbonLoadModel.getStorePath,
-                    carbonTable.getCarbonTableIdentifier))
-              val index = taskNo + 1
-              uniqueLoadStatusId = carbonLoadModel.getTableName +
-                                   CarbonCommonConstants.UNDERSCORE +
-                                   (index + "_0")
-
-              // convert timestamp
-              val timeStampInLong = updateModel.get.updatedTimeStamp + ""
-              loadMetadataDetails.setPartitionCount(partitionID)
-              loadMetadataDetails.setLoadName(segId)
-              loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_FAILURE)
-              carbonLoadModel.setPartitionId(partitionID)
-              carbonLoadModel.setSegmentId(segId)
-              carbonLoadModel.setTaskNo(String.valueOf(index))
-              carbonLoadModel.setFactTimeStamp(updateModel.get.updatedTimeStamp)
-
-              // During Block Spill case Increment of File Count and proper adjustment of Block
-              // naming is only done when AbstractFactDataWriter.java : initializeWriter get
-              // CarbondataFileName as null. For handling Block Spill not setting the
-              // CarbondataFileName in case of Update.
-              // carbonLoadModel.setCarbondataFileName(newBlockName)
-
-              // storeLocation = CarbonDataLoadRDD.initialize(carbonLoadModel, index)
-              loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS)
-              val rddIteratorKey = CarbonCommonConstants.RDDUTIL_UPDATE_KEY +
-                                   UUID.randomUUID().toString
-
-              try {
-                RddInpututilsForUpdate.put(rddIteratorKey,
-                  new RddIteratorForUpdate(iter, carbonLoadModel))
-                carbonLoadModel.setRddIteratorKey(rddIteratorKey)
-                CarbonDataLoadForUpdate
-                  .run(carbonLoadModel, index, storePath, kettleHomePath,
-                    segId, loadMetadataDetails, executionErrors)
-              } finally {
-                RddInpututilsForUpdate.remove(rddIteratorKey)
-              }
-            } catch {
-              case e: Exception =>
-                LOGGER.info("DataLoad failure")
-                LOGGER.error(e)
-                throw e
-            }
-
-            var finished = false
-
-            override def hasNext: Boolean = !finished
-
-            override def next(): (String, (LoadMetadataDetails, ExecutionErrors)) = {
-              finished = true
-              rddResult
-                .getKey(uniqueLoadStatusId,
-                  (loadMetadataDetails, executionErrors))
-            }
-          }
-          resultIter
-        }
-
-        val updateRdd = dataFrame.get.rdd
-
-
-        val keyRDD = updateRdd.map(row =>
-          // splitting as (key, value) i.e., (segment, updatedRows)
-          (row.get(row.size - 1).toString, Row(row.toSeq.slice(0, row.size - 1): _*))
-        )
-        val groupBySegmentRdd = keyRDD.groupByKey()
-
-        val nodeNumOfData = groupBySegmentRdd.partitions.flatMap[String, Array[String]] { p =>
-          DataLoadPartitionCoalescer.getPreferredLocs(groupBySegmentRdd, p).map(_.host)
-        }.distinct.size
-        val nodes = DistributionUtil.ensureExecutorsByNumberAndGetNodeList(nodeNumOfData,
-          sqlContext.sparkContext)
-        val groupBySegmentAndNodeRdd =
-          new UpdateCoalescedRDD[(String, scala.Iterable[Row])](groupBySegmentRdd,
-            nodes.distinct.toArray)
-
-        res = groupBySegmentAndNodeRdd.map(x =>
-          triggerDataLoadForSegment(x._1, x._2.toIterator).toList
-        ).collect()
-
-      }
-
       if (!updateModel.isDefined) {
       CarbonLoaderUtil.checkAndCreateCarbonDataLocation(storePath,
         carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName, currentLoadCount.toString)
@@ -798,9 +658,7 @@ object CarbonDataRDDFactory {
       var errorMessage: String = "DataLoad failure"
       var executorMessage: String = ""
       try {
-        if (updateModel.isDefined) {
-          loadDataFrameForUpdate()
-        } else if (dataFrame.isDefined) {
+        if (dataFrame.isDefined) {
           loadDataFrame()
         }
         else {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 63cf9de..e6efeaa 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
@@ -190,7 +190,6 @@ class CarbonDataFrameWriter(sqlContext: SQLContext, val dataFrame: DataFrame) {
        | LOAD DATA INPATH '$csvFolder'
        | INTO TABLE ${options.dbName}.${options.tableName}
        | OPTIONS ('FILEHEADER' = '${dataFrame.columns.mkString(",")}',
-       | 'USE_KETTLE' = '${options.useKettle}',
        | 'SINGLE_PASS' = '${options.singlePass}')
      """.stripMargin
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
index 002b6f8..4bd0564 100644
--- 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
@@ -110,8 +110,6 @@ case class AlterTableCompaction(alterTableModel: AlterTableModel) extends Runnab
     carbonLoadModel.setDatabaseName(relation.tableMeta.carbonTableIdentifier.getDatabaseName)
     carbonLoadModel.setStorePath(relation.tableMeta.storePath)
 
-    val kettleHomePath = CarbonScalaUtil.getKettleHome(sparkSession.sqlContext)
-
     var storeLocation = CarbonProperties.getInstance
       .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
         System.getProperty("java.io.tmpdir")
@@ -123,7 +121,6 @@ case class AlterTableCompaction(alterTableModel: AlterTableModel) extends Runnab
           alterTableModel,
           carbonLoadModel,
           relation.tableMeta.storePath,
-          kettleHomePath,
           storeLocation
         )
     } catch {
@@ -388,29 +385,6 @@ case class LoadTable(
 
       val columnar = sparkSession.conf.get("carbon.is.columnar.storage", "true").toBoolean
 
-      // TODO It will be removed after kettle is removed.
-      val useKettle = options.get("use_kettle") match {
-        case Some(value) => value.toBoolean
-        case _ =>
-          var useKettleLocal = System.getProperty("use_kettle")
-          if (useKettleLocal == null && sparkSession.sparkContext.getConf.contains("use_kettle")) {
-            useKettleLocal = sparkSession.sparkContext.getConf.get("use_kettle")
-          }
-          if (useKettleLocal == null) {
-            useKettleLocal = CarbonProperties.getInstance().
-              getProperty(CarbonCommonConstants.USE_KETTLE,
-                CarbonCommonConstants.USE_KETTLE_DEFAULT)
-          }
-          try {
-            useKettleLocal.toBoolean
-          } catch {
-            case e: Exception => CarbonCommonConstants.USE_KETTLE_DEFAULT.toBoolean
-          }
-      }
-
-      val kettleHomePath =
-        if (useKettle) CarbonScalaUtil.getKettleHome(sparkSession.sqlContext) else ""
-
       val delimiter = options.getOrElse("delimiter", ",")
       val quoteChar = options.getOrElse("quotechar", "\"")
       val fileHeader = options.getOrElse("fileheader", "")
@@ -455,11 +429,11 @@ case class LoadTable(
           DataLoadProcessorConstants.IS_EMPTY_DATA_BAD_RECORD + "," + isEmptyDataBadRecord)
       val useOnePass = options.getOrElse("single_pass", "false").trim.toLowerCase match {
         case "true" =>
-          if (!useKettle && StringUtils.isEmpty(allDictionaryPath)) {
+          if (StringUtils.isEmpty(allDictionaryPath)) {
             true
           } else {
             LOGGER.error("Can't use single_pass, because SINGLE_PASS and ALL_DICTIONARY_PATH" +
-              "can not be used together, and USE_KETTLE must be set as false")
+              "can not be used together")
             false
           }
         case "false" =>
@@ -539,10 +513,8 @@ case class LoadTable(
           CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
             carbonLoadModel,
             relation.tableMeta.storePath,
-            kettleHomePath,
             columnar,
             partitionStatus,
-            useKettle,
             result,
             dataFrame,
             updateModel)
@@ -590,10 +562,8 @@ case class LoadTable(
           CarbonDataRDDFactory.loadCarbonData(sparkSession.sqlContext,
             carbonLoadModel,
             relation.tableMeta.storePath,
-            kettleHomePath,
             columnar,
             partitionStatus,
-            useKettle,
             result,
             loadDataFrame,
             updateModel)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
index 15d5597..e94b6ed 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/test/Spark2TestQueryExecutor.scala
@@ -38,7 +38,6 @@ object Spark2TestQueryExecutor {
   private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
   LOGGER.info("use TestQueryExecutorImplV2")
   CarbonProperties.getInstance()
-    .addProperty("carbon.kettle.home", TestQueryExecutor.kettleHome)
     .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, TestQueryExecutor.timestampFormat)
     .addProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
       System.getProperty("java.io.tmpdir"))

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 bcc82ce..324d3a2 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
@@ -78,12 +78,6 @@ object TableLoader {
     System.out.println(s"table name: $dbName.$tableName")
     val inputPaths = TableAPIUtil.escape(args(2))
 
-    val kettleHome = CarbonProperties.getInstance().getProperty("carbon.kettle.home")
-    if (kettleHome == null) {
-      CarbonProperties.getInstance().addProperty("carbon.kettle.home",
-        map.getOrElse("carbon.kettle.home", ""))
-    }
-
     val spark = TableAPIUtil.spark(storePath, s"TableLoader: $dbName.$tableName")
 
     CarbonEnv.init(spark)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 33f710b..642b330 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
@@ -54,7 +54,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
            OPTIONS("bucketnumber"="4", "bucketcolumns"="name", "tableName"="t4")
       """)
     LoadTable(Some("default"), "t4", s"$resourcesPath/source.csv", Nil,
-      Map(("use_kettle", "false"))).run(sqlContext.sparkSession)
+      Map()).run(sqlContext.sparkSession)
     val table: CarbonTable = CarbonMetadata.getInstance().getCarbonTable("default_t4")
     if (table != null && table.getBucketingInfo("t4") != null) {
       assert(true)
@@ -90,7 +90,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
            OPTIONS("tableName"="t5")
       """)
     LoadTable(Some("default"), "t5", s"$resourcesPath/source.csv", Nil,
-      Map(("use_kettle", "false"))).run(sqlContext.sparkSession)
+      Map()).run(sqlContext.sparkSession)
 
     val plan = sql(
       """
@@ -115,7 +115,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
            OPTIONS("bucketnumber"="4", "bucketcolumns"="name", "tableName"="t6")
       """)
     LoadTable(Some("default"), "t6", s"$resourcesPath/source.csv", Nil,
-      Map(("use_kettle", "false"))).run(sqlContext.sparkSession)
+      Map()).run(sqlContext.sparkSession)
 
     val plan = sql(
       """
@@ -140,7 +140,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
            OPTIONS("bucketnumber"="4", "bucketcolumns"="name", "tableName"="t7")
       """)
     LoadTable(Some("default"), "t7", s"$resourcesPath/source.csv", Nil,
-      Map(("use_kettle", "false"))).run(sqlContext.sparkSession)
+      Map()).run(sqlContext.sparkSession)
 
     sql("DROP TABLE IF EXISTS bucketed_parquet_table")
     sql("select * from t7").write
@@ -171,7 +171,7 @@ class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
            OPTIONS("bucketnumber"="4", "bucketcolumns"="name", "tableName"="t8")
       """)
     LoadTable(Some("default"), "t8", s"$resourcesPath/source.csv", Nil,
-      Map(("use_kettle", "false"))).run(sqlContext.sparkSession)
+      Map()).run(sqlContext.sparkSession)
 
     sql("DROP TABLE IF EXISTS parquet_table")
     sql("select * from t8").write

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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 ceb340e..55eaa20 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
@@ -44,7 +44,7 @@ class VectorReaderTestCase extends QueryTest with BeforeAndAfterAll {
           OPTIONS("tableName"="vectorreader")
       """)
     LoadTable(Some("default"), "vectorreader", s"$resourcesPath/source.csv", Nil,
-      Map(("use_kettle", "false"))).run(sqlContext.sparkSession)
+      Map()).run(sqlContext.sparkSession)
   }
 
   test("test vector reader") {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 95a3ff4..a132d6d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,8 +105,6 @@
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
     <snappy.version>1.1.2.6</snappy.version>
     <hadoop.version>2.2.0</hadoop.version>
-    <kettle.version>4.4.0-stable</kettle.version>
-    <use.kettle>false</use.kettle>
     <hadoop.deps.scope>compile</hadoop.deps.scope>
     <spark.deps.scope>compile</spark.deps.scope>
     <scala.deps.scope>compile</scala.deps.scope>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/kettle.properties
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/kettle.properties b/processing/carbonplugins/.kettle/kettle.properties
deleted file mode 100644
index 9bae2e3..0000000
--- a/processing/carbonplugins/.kettle/kettle.properties
+++ /dev/null
@@ -1,10 +0,0 @@
-# This file was generated by Pentaho Data Integration version 4.2.1.
-#
-# Here are a few examples of variables to set:
-#
-# PRODUCTION_SERVER = hercules
-# TEST_SERVER = zeus
-# DEVELOPMENT_SERVER = thor
-#
-# Note: lines like these with a # in front of it are comments
-#

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbonaggregatesurrogategenerator/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonaggregatesurrogategenerator/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonaggregatesurrogategenerator/plugin.xml
deleted file mode 100644
index 2ae5a87..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbonaggregatesurrogategenerator/plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="CarbonAggSurrogateGenerator"
-   description="Carbon Agg Surrogate Generator"
-   tooltip="Carbon Agg Surrogate Generator"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.aggregatesurrogategenerator.step.CarbonAggregateSurrogateGeneratorMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbonautoagggraphgenerator/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonautoagggraphgenerator/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonautoagggraphgenerator/plugin.xml
deleted file mode 100644
index fc3cdf5..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbonautoagggraphgenerator/plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="CarbonAutoAggGraphGenerator"
-   description="Carbon Auto Agg Graph Generator"
-   tooltip="Carbon Auto Agg Graph Generator"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.autoaggregategraphgenerator.step.CarbonAutoAGGGraphGeneratorMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbonautoaggslicemerger/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonautoaggslicemerger/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonautoaggslicemerger/plugin.xml
deleted file mode 100644
index a0a2192..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbonautoaggslicemerger/plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="CarbonAutoAggSliceMerger"
-   description="Carbon Auto Agg Slice Merger"
-   tooltip="Carbon Auto Agg Slice Merger"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.merger.step.autoaggregate.CarbonAutoAggregateSliceMergerMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carboncsvbasedseqgen/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carboncsvbasedseqgen/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carboncsvbasedseqgen/plugin.xml
deleted file mode 100644
index 91617c5..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carboncsvbasedseqgen/plugin.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-
-   id="CarbonCSVBasedSurrogateGen"
-   description="Carbon CSV Based Surrogate Key Generator"
-   tooltip="Carbon CSV based Surrogate Key Generator"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.CarbonCSVBasedSeqGenMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carboncsvreader/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carboncsvreader/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carboncsvreader/plugin.xml
deleted file mode 100644
index 7f33b5e..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carboncsvreader/plugin.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-
-   id="CSVReader"
-   description="Carbon CSVReader"
-   tooltip="Carbon CSVReader"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.csvreader.CsvReaderMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carboncsvreaderstrep/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carboncsvreaderstrep/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carboncsvreaderstrep/plugin.xml
deleted file mode 100644
index 7173171..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carboncsvreaderstrep/plugin.xml
+++ /dev/null
@@ -1,29 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-
-   id="CarbonCSVReaderStep"
-   description="Carbon CSV Reader Step"
-   tooltip="Carbon CSV Reader Step"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.csvreaderstep.CsvInputMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbondatawriter/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbondatawriter/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbondatawriter/plugin.xml
deleted file mode 100644
index 92631a1..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbondatawriter/plugin.xml
+++ /dev/null
@@ -1,27 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="CarbonDataWriter"
-   description="Carbon Data Writer"
-   tooltip="Carbon Data Writer"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.store.CarbonDataWriterStepMeta">
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbonfactreader/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonfactreader/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonfactreader/plugin.xml
deleted file mode 100644
index 270bbe3..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbonfactreader/plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="CarbonFactReader"
-   description="Carbon Fact Reader"
-   tooltip="Carbon Fact Reader"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.factreader.step.CarbonFactReaderMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbongroupby/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbongroupby/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbongroupby/plugin.xml
deleted file mode 100644
index 087589e..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbongroupby/plugin.xml
+++ /dev/null
@@ -1,27 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="CarbonGroupBy"
-   description="Carbon Group By"
-   tooltip="Carbon Group By"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.groupby.step.CarbonGroupByStepMeta">
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carboninmemoryfactreader/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carboninmemoryfactreader/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carboninmemoryfactreader/plugin.xml
deleted file mode 100644
index 550531c..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carboninmemoryfactreader/plugin.xml
+++ /dev/null
@@ -1,27 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="CarbonInMemoryFactReader"
-   description="Carbon InMemory Fact Reader"
-   tooltip="Carbon InMemory Fact Reader"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.factreader.inmemory.step.CarbonInMemoryFactReaderMeta">
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbonseqgen/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonseqgen/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonseqgen/plugin.xml
deleted file mode 100644
index 517b757..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbonseqgen/plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-
-   id="CarbonSurrogateGen"
-   description="Carbon Surrogate Key Generator"
-   tooltip="Carbon Surrogate Key Generator"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.surrogatekeysgenerator.dbbased.CarbonSeqGenStepMeta">
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbonslicemerger/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonslicemerger/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonslicemerger/plugin.xml
deleted file mode 100644
index 618dba0..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbonslicemerger/plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-
-   id="CarbonSliceMerger"
-   description="Carbon Slice Merger"
-   tooltip="Carbon Slice Merger"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.merger.step.CarbonSliceMergerStepMeta">
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/carbonsortkeyandgroupby/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/carbonsortkeyandgroupby/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/carbonsortkeyandgroupby/plugin.xml
deleted file mode 100644
index 970a855..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/carbonsortkeyandgroupby/plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="CarbonSortKeyGroupBy"
-   description="Carbon Sort And Group By"
-   tooltip="Carbon Sort And Group By"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.sortandgroupby.step.CarbonSortKeyAndGroupByStepMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/mdkeygenstep/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/mdkeygenstep/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/mdkeygenstep/plugin.xml
deleted file mode 100644
index e0892a4..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/mdkeygenstep/plugin.xml
+++ /dev/null
@@ -1,28 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="MDKeyGen"
-   description="MD Key Gen"
-   tooltip="MD Key Gen"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.mdkeygen.MDKeyGenStepMeta">
-
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/carbonplugins/.kettle/plugins/steps/sortkeystep/plugin.xml
----------------------------------------------------------------------
diff --git a/processing/carbonplugins/.kettle/plugins/steps/sortkeystep/plugin.xml b/processing/carbonplugins/.kettle/plugins/steps/sortkeystep/plugin.xml
deleted file mode 100644
index d08d0ee..0000000
--- a/processing/carbonplugins/.kettle/plugins/steps/sortkeystep/plugin.xml
+++ /dev/null
@@ -1,27 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one
- ~ or more contributor license agreements.  See the NOTICE file
- ~ distributed with this work for additional information
- ~ regarding copyright ownership.  The ASF licenses this file
- ~ to you under the Apache License, Version 2.0 (the
- ~ "License"); you may not use this file except in compliance
- ~ with the License.  You may obtain a copy of the License at
- ~
- ~    http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing,
- ~ software distributed under the License is distributed on an
- ~ "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- ~ KIND, either express or implied.  See the License for the
- ~ specific language governing permissions and limitations
- ~ under the License.
- -->
-
-<plugin
-   id="SortKey"
-   description="Carbon Sort"
-   tooltip="Carbon Sort"
-   category="Carbon"
-   classname="org.apache.carbondata.processing.sortandgroupby.sortdatastep.SortKeyStepMeta">
-</plugin>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/pom.xml
----------------------------------------------------------------------
diff --git a/processing/pom.xml b/processing/pom.xml
index 096f49a..57b0908 100644
--- a/processing/pom.xml
+++ b/processing/pom.xml
@@ -49,37 +49,11 @@
       <groupId>org.apache.spark</groupId>
       <artifactId>spark-sql_${scala.binary.version}</artifactId>
     </dependency>
-    <dependency>
-      <groupId>pentaho-kettle</groupId>
-      <artifactId>kettle-engine</artifactId>
-      <version>${kettle.version}</version>
-    </dependency>
 	<dependency>
       <groupId>com.univocity</groupId>
       <artifactId>univocity-parsers</artifactId>
       <version>1.5.6</version>
     </dependency>
-	<dependency>
-      <groupId>pentaho-kettle</groupId>
-      <artifactId>kettle-core</artifactId>
-      <version>${kettle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>pentaho-kettle</groupId>
-      <artifactId>kettle-db</artifactId>
-      <version>${kettle.version}</version>
-    </dependency>
-    <dependency>
-      <groupId>commons-vfs</groupId>
-      <artifactId>commons-vfs</artifactId>
-      <version>1.0</version>
-      <exclusions>
-        <exclusion>
-          <groupId>*</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-lang3</artifactId>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
new file mode 100644
index 0000000..d6d214b
--- /dev/null
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/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.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/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index 7e6f6f4..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
+++ /dev/null
@@ -1,475 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-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.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.processing.api.dataloader.SchemaInfo;
-import org.apache.carbondata.processing.constants.DataProcessorConstants;
-import org.apache.carbondata.processing.csvreaderstep.CsvInputMeta;
-import org.apache.carbondata.processing.dataprocessor.IDataProcessStatus;
-import org.apache.carbondata.processing.etl.DataLoadingException;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordsLogger;
-import org.apache.carbondata.processing.util.CarbonDataProcessorUtil;
-
-import org.pentaho.di.core.KettleEnvironment;
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.logging.LogLevel;
-import org.pentaho.di.core.logging.LoggingObjectInterface;
-import org.pentaho.di.core.logging.LoggingRegistry;
-import org.pentaho.di.core.xml.XMLHandlerCache;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.steps.getfilenames.GetFileNamesMeta;
-import org.pentaho.di.trans.steps.hadoopfileinput.HadoopFileInputMeta;
-import org.pentaho.di.trans.steps.textfileinput.TextFileInputField;
-
-public class DataGraphExecuter {
-  /**
-   * Comment for <code>LOGGER</code>
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DataGraphExecuter.class.getName());
-  /**
-   * graph transformation object
-   */
-  private Trans trans;
-  /**
-   *
-   */
-  private IDataProcessStatus model;
-
-  public DataGraphExecuter(IDataProcessStatus model) {
-    this.model = model;
-  }
-
-  /**
-   * This Method checks whether csv file provided and the column name in schema are same
-   * or not
-   *
-   * @param columnNames
-   * @param csvFilePath
-   * @return true if same, false otherwise.
-   */
-  private boolean checkCSVAndRequestedTableColumns(String[] columnNames, String csvFilePath,
-      String delimiter) throws IOException {
-    return GraphExecutionUtil.checkCSVAndRequestedTableColumns(csvFilePath, columnNames, delimiter);
-  }
-
-  /**
-   * This method returns the Columns names from the schema.
-   *
-   * @param tableName
-   * @return column names array.
-   */
-  private String[] getColumnNames(String tableName, CarbonDataLoadSchema schema) {
-    Set<String> columnNames = CarbonDataProcessorUtil.getSchemaColumnNames(schema, tableName);
-    return columnNames.toArray(new String[columnNames.size()]);
-  }
-
-  private void validateCSV(String tableName, CarbonFile f, CarbonDataLoadSchema schema,
-      String delimiter) throws DataLoadingException, IOException {
-
-    String[] columnNames = getColumnNames(tableName, schema);
-
-    if (!checkCSVAndRequestedTableColumns(columnNames, f.getAbsolutePath(), delimiter)) {
-      LOGGER.error(
-          "CSV File provided is not proper. Column names in schema and csv header are not same. "
-              + "CSVFile Name : "
-              + f.getName());
-      throw new DataLoadingException(DataProcessorConstants.CSV_VALIDATION_ERRROR_CODE,
-          "CSV File provided is not proper. Column names in schema and csv header are not same. "
-              + "CSVFile Name : "
-              + f.getName());
-    }
-  }
-
-  public void executeGraph(String graphFilePath, SchemaInfo schemaInfo, CarbonDataLoadSchema schema)
-      throws DataLoadingException {
-
-    //This Method will validate the both fact and dimension csv files.
-    if (!schemaInfo.isAutoAggregateRequest() && model.getRddIteratorKey() == null) {
-      validateCSVFiles(schema);
-    }
-    execute(graphFilePath, schemaInfo);
-  }
-
-  /**
-   * executeGraph which generate the kettle graph
-   *
-   * @throws DataLoadingException
-   */
-
-  private void execute(String graphFilePath, SchemaInfo schemaInfo)
-      throws DataLoadingException {
-
-    //This Method will validate the both fact and dimension csv files.
-
-    initKettleEnv();
-    TransMeta transMeta = null;
-    try {
-      transMeta = new TransMeta(graphFilePath);
-      transMeta.setFilename(graphFilePath);
-      trans = new Trans(transMeta);
-      if (!schemaInfo.isAutoAggregateRequest()) {
-        // Register HDFS as a file system type with VFS to make HadoopFileInputMeta work
-        boolean hdfsReadMode =
-            model.getCsvFilePath() != null && model.getCsvFilePath().startsWith("hdfs:");
-        trans.setVariable("modifiedDimNames", model.getDimTables());
-        trans.setVariable("csvInputFilePath", model.getCsvFilePath());
-        trans.setVariable(CarbonCommonConstants.BAD_RECORD_KEY, null);
-        if (hdfsReadMode) {
-          trans.addParameterDefinition("vfs.hdfs.dfs.client.read.shortcircuit", "true", "");
-          trans.addParameterDefinition("vfs.hdfs.dfs.domain.socket.path",
-              "/var/lib/hadoop-hdfs-new/dn_socket", "");
-          trans.addParameterDefinition("vfs.hdfs.dfs.block.local-path-access.user", "hadoop,root",
-              "");
-          trans.addParameterDefinition("vfs.hdfs.io.file.buffer.size", "5048576", "");
-        }
-        List<StepMeta> stepsMeta = trans.getTransMeta().getSteps();
-        StringBuilder builder = new StringBuilder();
-        StringBuilder measuresInCSVFile = new StringBuilder();
-        processCsvInputMeta(stepsMeta, builder, measuresInCSVFile);
-        processGetFileNamesMeta(stepsMeta);
-
-        processHadoopFileInputMeta(stepsMeta, builder, measuresInCSVFile);
-      }
-      setGraphLogLevel();
-      trans.execute(null);
-      LOGGER.info("Graph execution is started " + graphFilePath);
-      trans.waitUntilFinished();
-      LOGGER.info("Graph execution is finished.");
-    } catch (KettleException | IOException e) {
-      LOGGER.error(e, "Unable to start execution of graph " + e.getMessage());
-      throw new DataLoadingException("Unable to start execution of graph ", e);
-    }
-
-    //Don't change the logic of creating key
-    String key = model.getDatabaseName() + '/' + model.getTableName() + '_' + model.getTableName();
-
-    if (trans.getErrors() > 0) {
-      if (null != trans.getVariable(CarbonCommonConstants.BAD_RECORD_KEY)) {
-        LOGGER.error(trans.getVariable(CarbonCommonConstants.BAD_RECORD_KEY));
-        throw new DataLoadingException(
-            "Data load failed due to bad record ," + trans
-                .getVariable(CarbonCommonConstants.BAD_RECORD_KEY));
-      }
-      LOGGER.error("Graph Execution had errors");
-      throw new DataLoadingException("Due to internal errors, please check logs for more details.");
-    } else if (null != BadRecordsLogger.hasBadRecord(key)) {
-      LOGGER.error("Data load is partially success");
-      throw new DataLoadingException(DataProcessorConstants.BAD_REC_FOUND,
-          "Data load is partially success");
-    } else {
-      LOGGER.info("Graph execution task is over with No error.");
-    }
-    LoggingRegistry instance = LoggingRegistry.getInstance();
-    Map<String, LoggingObjectInterface> map = instance.getMap();
-    if (null != map) {
-      for (Entry<String, LoggingObjectInterface> entry : map.entrySet()) {
-        instance.removeIncludingChildren(entry.getKey());
-      }
-    }
-
-    map = null;
-    XMLHandlerCache.getInstance().clear();
-    trans.cleanup();
-    trans.eraseParameters();
-    trans.killAll();
-    trans = null;
-  }
-
-  /**
-   * @param stepsMeta
-   * @param builder
-   * @param measuresInCSVFile
-   * @throws DataLoadingException
-   */
-  private void processHadoopFileInputMeta(List<StepMeta> stepsMeta, StringBuilder builder,
-      StringBuilder measuresInCSVFile) throws DataLoadingException {
-    for (StepMeta step : stepsMeta) {
-      if (step.getStepMetaInterface() instanceof HadoopFileInputMeta) {
-
-        HadoopFileInputMeta stepMetaInterface = (HadoopFileInputMeta) step.getStepMetaInterface();
-        if (null != model.getCsvFilePath()) {
-          stepMetaInterface.setFilenameField("filename");
-          stepMetaInterface.setFileName(new String[] { "${csvInputFilePath}" });
-          stepMetaInterface.setDefault();
-          stepMetaInterface.setEncoding(CarbonCommonConstants.DEFAULT_CHARSET);
-          stepMetaInterface.setEnclosure("\"");
-          stepMetaInterface.setHeader(true);
-          stepMetaInterface.setSeparator(",");
-          stepMetaInterface.setAcceptingFilenames(true);
-          stepMetaInterface.setAcceptingStepName("getFileNames");
-          stepMetaInterface.setFileFormat("mixed");
-          stepMetaInterface.setAcceptingField("filename");
-
-          CarbonFile csvFileToRead = GraphExecutionUtil.getCsvFileToRead(model.getCsvFilePath());
-          TextFileInputField[] inputFields = GraphExecutionUtil
-              .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile, ",");
-          stepMetaInterface.setInputFields(inputFields);
-        } else if (model.isDirectLoad()) {
-          String[] files = new String[model.getFilesToProcess().size()];
-          int i = 0;
-          for (String file : model.getFilesToProcess()) {
-            files[i++] = file;
-          }
-          stepMetaInterface.setFileName(files);
-          stepMetaInterface.setFilenameField("filename");
-          stepMetaInterface.setDefault();
-          stepMetaInterface.setEncoding(CarbonCommonConstants.DEFAULT_CHARSET);
-          stepMetaInterface.setEnclosure("\"");
-          stepMetaInterface.setHeader(true);
-          stepMetaInterface.setSeparator(",");
-          stepMetaInterface.setAcceptingFilenames(true);
-          stepMetaInterface.setAcceptingStepName("getFileNames");
-          stepMetaInterface.setFileFormat("mixed");
-          stepMetaInterface.setAcceptingField("filename");
-
-          if (null != model.getCsvHeader() && !model.getCsvHeader().isEmpty()) {
-            TextFileInputField[] inputParams = GraphExecutionUtil
-                .getTextInputFiles(model.getCsvHeader(), builder, measuresInCSVFile, ",");
-            ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputParams);
-            ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
-            ((CsvInputMeta) step.getStepMetaInterface())
-              .setEscapeCharacter(model.getEscapeCharacter());
-            ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(false);
-
-          } else if (model.getFilesToProcess().size() > 0) {
-            CarbonFile csvFile =
-                GraphExecutionUtil.getCsvFileToRead(model.getFilesToProcess().get(0));
-            TextFileInputField[] inputFields = GraphExecutionUtil
-                .getTextInputFiles(csvFile, builder, measuresInCSVFile,
-                    model.getCsvDelimiter());
-            ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
-            ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
-            ((CsvInputMeta) step.getStepMetaInterface())
-              .setEscapeCharacter(model.getEscapeCharacter());
-            ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(true);
-          }
-        }
-
-        break;
-      }
-    }
-  }
-
-  /**
-   * @param stepsMeta
-   * @throws IOException
-   */
-  private void processGetFileNamesMeta(List<StepMeta> stepsMeta) throws IOException {
-    for (StepMeta step : stepsMeta) {
-      if (step.getStepMetaInterface() instanceof GetFileNamesMeta) {
-        GetFileNamesMeta stepMetaInterface = (GetFileNamesMeta) step.getStepMetaInterface();
-        if (null != model.getCsvFilePath()) {
-          boolean checkIsFolder = GraphExecutionUtil.checkIsFolder(model.getCsvFilePath());
-          if (checkIsFolder) {
-            stepMetaInterface.setFileName(new String[] { model.getCsvFilePath() });
-            stepMetaInterface.setFileMask(new String[] { ".*\\.csv$|.*\\.inprogress" });
-            stepMetaInterface.setExcludeFileMask(new String[] { "1" });
-          } else {
-            //If absolute file path is provided for the data load and stopped in between then csv
-            // file will be
-            // changed to inprogress, and when next time server start then we need to check the
-            // file name extension.
-            // can contain .csv.inprogress file.
-
-            FileType fileType = FileFactory.getFileType(model.getCsvFilePath());
-
-            boolean exists = FileFactory.isFileExist(model.getCsvFilePath(), fileType);
-
-            if (exists) {
-              stepMetaInterface.setFileName(new String[] { model.getCsvFilePath() });
-              stepMetaInterface.setExcludeFileMask(new String[] { null });
-            } else {
-              stepMetaInterface.setFileName(new String[] {
-                  model.getCsvFilePath() + CarbonCommonConstants.FILE_INPROGRESS_STATUS });
-              stepMetaInterface.setExcludeFileMask(new String[] { null });
-            }
-          }
-        } else if (model.isDirectLoad()) {
-          String[] files = new String[model.getFilesToProcess().size()];
-          int i = 0;
-          for (String file : model.getFilesToProcess()) {
-            files[i++] = file;
-          }
-          stepMetaInterface.setFileName(files);
-        }
-        break;
-      }
-    }
-  }
-
-  /**
-   * @param stepsMeta
-   * @param builder
-   * @param measuresInCSVFile
-   * @throws DataLoadingException
-   */
-  private void processCsvInputMeta(List<StepMeta> stepsMeta, StringBuilder builder,
-      StringBuilder measuresInCSVFile) throws DataLoadingException {
-    for (StepMeta step : stepsMeta) {
-      if (step.getStepMetaInterface() instanceof CsvInputMeta) {
-        if (null != model.getCsvFilePath() && model.getRddIteratorKey() == null) {
-          CarbonFile csvFileToRead = GraphExecutionUtil.getCsvFileToRead(model.getCsvFilePath());
-          TextFileInputField[] inputFields = GraphExecutionUtil
-              .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile, ",");
-          ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
-        } else if (model.isDirectLoad()) {
-          if (null != model.getCsvHeader() && !model.getCsvHeader().isEmpty()) {
-            TextFileInputField[] inputFields = GraphExecutionUtil
-                .getTextInputFiles(model.getCsvHeader(), builder, measuresInCSVFile, ",");
-            ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
-            ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
-            ((CsvInputMeta) step.getStepMetaInterface())
-                .setEscapeCharacter(model.getEscapeCharacter());
-            ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(false);
-
-          } else if (model.getFilesToProcess().size() > 0) {
-            CarbonFile csvFileToRead =
-                GraphExecutionUtil.getCsvFileToRead(model.getFilesToProcess().get(0));
-            TextFileInputField[] inputFields = GraphExecutionUtil
-                .getTextInputFiles(csvFileToRead, builder, measuresInCSVFile,
-                    model.getCsvDelimiter());
-            ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
-            ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
-            ((CsvInputMeta) step.getStepMetaInterface())
-              .setEscapeCharacter(model.getEscapeCharacter());
-            ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(true);
-          }
-        }
-        break;
-      }
-    }
-  }
-
-  /**
-   *
-   */
-  private void initKettleEnv() {
-    try {
-      KettleEnvironment.init(false);
-      LOGGER.info("Kettle environment initialized");
-    } catch (KettleException ke) {
-      LOGGER.error("Unable to initialize Kettle Environment " + ke.getMessage());
-    }
-  }
-
-
-  private void setGraphLogLevel() {
-    trans.setLogLevel(LogLevel.NOTHING);
-  }
-
-  /**
-   * This method will validate the both fact as well as dimension csv files.
-   *
-   * @param schema
-   * @throws DataLoadingException
-   */
-  private void validateCSVFiles(CarbonDataLoadSchema schema) throws DataLoadingException {
-    // Validate the Fact CSV Files.
-    String csvFilePath = model.getCsvFilePath();
-    if (csvFilePath != null) {
-      FileType fileType = FileFactory.getFileType(csvFilePath);
-      try {
-        boolean exists = FileFactory.isFileExist(csvFilePath, fileType);
-        if (exists && FileFactory.getCarbonFile(csvFilePath, fileType).isDirectory()) {
-          CarbonFile fileDir = FileFactory.getCarbonFile(csvFilePath, fileType);
-          CarbonFile[] listFiles = fileDir.listFiles(new CarbonFileFilter() {
-
-            @Override public boolean accept(CarbonFile pathname) {
-              if (pathname.getName().endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION) || pathname
-                  .getName().endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION
-                      + CarbonCommonConstants.FILE_INPROGRESS_STATUS)) {
-                return true;
-              }
-              return false;
-            }
-          });
-
-          for (CarbonFile f : listFiles) {
-            validateCSV(model.getTableName(), f, schema, ",");
-          }
-        } else {
-
-          if (!(csvFilePath.endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION) || csvFilePath
-              .endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION
-                  + CarbonCommonConstants.FILE_INPROGRESS_STATUS))) {
-            LOGGER.error("File provided is not proper, Only csv files are allowed." + csvFilePath);
-            throw new DataLoadingException(
-                "File provided is not proper, Only csv files are allowed." + csvFilePath);
-          }
-
-          if (exists) {
-            validateCSV(model.getTableName(),
-                FileFactory.getCarbonFile(csvFilePath, fileType), schema, ",");
-          } else {
-            validateCSV(model.getTableName(), FileFactory
-                .getCarbonFile(csvFilePath + CarbonCommonConstants.FILE_INPROGRESS_STATUS,
-                    fileType), schema, ",");
-          }
-
-        }
-
-      } catch (IOException e) {
-        LOGGER.error(e,
-            "Error while checking file exists" + csvFilePath);
-      }
-    } else if (model.isDirectLoad()) {
-      if (null != model.getCsvHeader() && !model.getCsvHeader().isEmpty()) {
-        if (!CarbonDataProcessorUtil
-            .isHeaderValid(model.getTableName(), model.getCsvHeader(), schema, ",")) {
-          LOGGER.error("CSV header provided in DDL is not proper."
-              + " Column names in schema and CSV header are not the same.");
-          throw new DataLoadingException(DataProcessorConstants.CSV_VALIDATION_ERRROR_CODE,
-              "CSV header provided in DDL is not proper. Column names in schema and CSV header are "
-                  + "not the same.");
-        }
-      } else {
-        for (String file : model.getFilesToProcess()) {
-          try {
-            FileFactory.FileType fileType = FileFactory.getFileType(file);
-            if (FileFactory.isFileExist(file, fileType)) {
-              validateCSV(model.getTableName(),
-                  FileFactory.getCarbonFile(file, fileType), schema,
-                  model.getCsvDelimiter());
-            }
-          } catch (IOException e) {
-            LOGGER.error(e,
-                "Error while checking file exists" + file);
-          }
-        }
-      }
-    }
-  }
-
-}


[09/13] incubator-carbondata git commit: Removed kettle related code and refactored

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index 5504ba8..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.BufferedReader;
-import java.io.DataInputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-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.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.util.CarbonUtil;
-import org.apache.carbondata.processing.etl.DataLoadingException;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema;
-import org.apache.carbondata.processing.model.CarbonDataLoadSchema.DimensionRelation;
-
-import org.pentaho.di.trans.steps.textfileinput.TextFileInputField;
-
-public final class GraphExecutionUtil {
-  /**
-   * Comment for <code>LOGGER</code>
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(GraphExecutionUtil.class.getName());
-
-  private GraphExecutionUtil() {
-
-  }
-
-  /**
-   * getCsvFileToRead
-   *
-   * @param csvFilePath
-   * @return File
-   */
-  public static CarbonFile getCsvFileToRead(String csvFilePath) {
-    CarbonFile csvFile =
-        FileFactory.getCarbonFile(csvFilePath, FileFactory.getFileType(csvFilePath));
-
-    CarbonFile[] listFiles = null;
-    if (csvFile.isDirectory()) {
-      listFiles = csvFile.listFiles(new CarbonFileFilter() {
-        @Override public boolean accept(CarbonFile pathname) {
-          if (!pathname.isDirectory()) {
-            if (pathname.getName().endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION) || pathname
-                .getName().endsWith(CarbonCommonConstants.CSV_FILE_EXTENSION
-                    + CarbonCommonConstants.FILE_INPROGRESS_STATUS)) {
-              return true;
-            }
-          }
-
-          return false;
-        }
-      });
-    } else {
-      listFiles = new CarbonFile[1];
-      listFiles[0] = csvFile;
-
-    }
-
-    return listFiles[0];
-  }
-
-  /**
-   * @param measuresInCSVFile
-   * @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;
-
-    FileType fileType = FileFactory.getFileType(csvFile.getAbsolutePath());
-
-    if (!csvFile.exists()) {
-      csvFile = FileFactory
-          .getCarbonFile(csvFile.getAbsolutePath() + CarbonCommonConstants.FILE_INPROGRESS_STATUS,
-              fileType);
-    }
-
-    try {
-      fileReader = FileFactory.getDataInputStream(csvFile.getAbsolutePath(), fileType);
-      bufferedReader =
-          new BufferedReader(new InputStreamReader(fileReader, Charset.defaultCharset()));
-      readLine = bufferedReader.readLine();
-    } catch (FileNotFoundException e) {
-      LOGGER.error(e, "CSV Input File not found  " + e.getMessage());
-      throw new DataLoadingException("CSV Input File not found ", e);
-    } catch (IOException e) {
-      LOGGER.error(e, "Not able to read CSV input File  " + e.getMessage());
-      throw new DataLoadingException("Not able to read CSV input File ", e);
-    } finally {
-      CarbonUtil.closeStreams(fileReader, bufferedReader);
-    }
-
-    if (null != readLine) {
-      delimiter = CarbonUtil.delimiterConverter(delimiter);
-      String[] columnNames = readLine.split(delimiter);
-      TextFileInputField[] textFileInputFields = new TextFileInputField[columnNames.length];
-
-      int i = 0;
-      String tmpCol;
-      for (String column : columnNames) {
-        tmpCol = column.replaceAll("\"", "");
-        builder.append(tmpCol);
-        builder.append(";");
-        textFileInputFields[i] = new TextFileInputField();
-        textFileInputFields[i].setName(tmpCol.trim());
-        textFileInputFields[i].setType(2);
-        measuresInCSVFile.append(tmpCol);
-        measuresInCSVFile.append(";");
-        i++;
-      }
-
-      return textFileInputFields;
-    }
-
-    return null;
-  }
-
-  /**
-   * @param measuresInCSVFile
-   * @throws DataLoadingException
-   */
-  public static TextFileInputField[] getTextInputFiles(String header, StringBuilder builder,
-      StringBuilder measuresInCSVFile, String delimiter) throws DataLoadingException {
-
-    String[] columnNames = header.split(delimiter);
-    TextFileInputField[] textFileInputFields = new TextFileInputField[columnNames.length];
-
-    int i = 0;
-    String tmpCol;
-    for (String columnName : columnNames) {
-      tmpCol = columnName.replaceAll("\"", "");
-      builder.append(tmpCol);
-      builder.append(";");
-      textFileInputFields[i] = new TextFileInputField();
-      textFileInputFields[i].setName(tmpCol.trim());
-      textFileInputFields[i].setType(2);
-      measuresInCSVFile.append(tmpCol);
-      measuresInCSVFile.append(";");
-      i++;
-    }
-
-    return textFileInputFields;
-
-  }
-
-  public static boolean checkIsFolder(String csvFilePath) {
-    try {
-      if (FileFactory.isFileExist(csvFilePath, FileFactory.getFileType(csvFilePath), false)) {
-        CarbonFile carbonFile =
-            FileFactory.getCarbonFile(csvFilePath, FileFactory.getFileType(csvFilePath));
-        return carbonFile.isDirectory();
-      }
-    } catch (IOException e) {
-      LOGGER.error(e,
-          "Not able check path exists or not  " + e.getMessage() + "path: " + csvFilePath);
-    }
-
-    return false;
-  }
-
-  /**
-   * @param csvFilePath
-   * @param columnNames
-   * @return
-   */
-  public static boolean checkCSVAndRequestedTableColumns(String csvFilePath, String[] columnNames,
-      String delimiter) throws IOException {
-
-    String readLine = CarbonUtil.readHeader(csvFilePath);
-
-    if (null != readLine) {
-      delimiter = CarbonUtil.delimiterConverter(delimiter);
-      String[] columnFromCSV = readLine.toLowerCase().split(delimiter);
-
-      List<String> csvColumnsList = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-      for (String column : columnFromCSV) {
-        csvColumnsList.add(column.replaceAll("\"", "").trim());
-      }
-
-      int count = 0;
-
-      for (String columns : columnNames) {
-        if (csvColumnsList.contains(columns.toLowerCase())) {
-          count++;
-        }
-      }
-      if (0 == count) {
-        LOGGER.error("There is No proper CSV file header found." +
-            " Either the ddl or the CSV file should provide CSV file header. ");
-      }
-      return (count == columnNames.length);
-    }
-
-    return false;
-  }
-
-  public static Set<String> getDimensionColumnNames(String dimTableName,
-      CarbonDataLoadSchema schema) {
-    Set<String> columnNames = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    for (DimensionRelation dimRel : schema.getDimensionRelationList()) {
-      if (dimRel.getTableName().equals(dimTableName)) {
-        for (String field : dimRel.getColumns()) {
-          columnNames.add(field);
-        }
-        break;
-      }
-    }
-    return columnNames;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BlockDetails.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BlockDetails.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BlockDetails.java
deleted file mode 100644
index 328c3b7..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BlockDetails.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.csvreaderstep;
-
-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
- * TODO Remove this class after removing of kettle.
- */
-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/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BoundedDataStream.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BoundedDataStream.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BoundedDataStream.java
deleted file mode 100644
index 94bef1e..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/BoundedDataStream.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.processing.csvreaderstep;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * Custom reader class to read the data from file it will take care of reading
- * till the limit assigned to this class
- */
-public class BoundedDataStream 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 BoundedDataStream(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();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/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
deleted file mode 100644
index 6b65da0..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
+++ /dev/null
@@ -1,431 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.csvreaderstep;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-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 org.apache.carbondata.common.CarbonIterator;
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.common.logging.impl.StandardLogService;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory;
-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.row.RowMeta;
-import org.pentaho.di.core.row.ValueMetaInterface;
-import org.pentaho.di.i18n.BaseMessages;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStep;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInterface;
-import org.pentaho.di.trans.steps.textfileinput.EncodingType;
-
-/**
- * Read a simple CSV file
- * Just output Strings found in the file...
- */
-public class CsvInput extends BaseStep implements StepInterface {
-  private static final Class<?> PKG = CsvInput.class;
-  // for i18n purposes, needed by Translator2!!   $NON-NLS-1$
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CsvInput.class.getName());
-  /**
-   * NUM_CORES_DEFAULT_VAL
-   */
-  private static final int NUM_CORES_DEFAULT_VAL = 2;
-  /**
-   * ReentrantLock getFileBlockLock
-   */
-  private final Object getBlockListLock = new Object();
-  /**
-   * ReentrantLock putRowLock
-   */
-  private final Object putRowLock = new Object();
-  private CsvInputMeta meta;
-  private CsvInputData data;
-  /**
-   * resultArray
-   */
-  private Future[] resultArray;
-  private List<List<BlockDetails>> threadBlockList = new ArrayList<>();
-
-  private ExecutorService exec;
-
-  /**
-   * If rddIteratorKey is not null, read data from RDD
-   */
-  private String rddIteratorKey = null;
-
-  private CarbonIterator<CarbonIterator<String[]>> rddIterator;
-
-  public CsvInput(StepMeta stepMeta, StepDataInterface stepDataInterface, int copyNr,
-      TransMeta transMeta, Trans trans) {
-    super(stepMeta, stepDataInterface, copyNr, transMeta, trans);
-    LOGGER.info("** Using csv file **");
-  }
-
-  public boolean processRow(StepMetaInterface smi, StepDataInterface sdi) throws KettleException {
-    meta = (CsvInputMeta) smi;
-    data = (CsvInputData) sdi;
-
-    if (first) {
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordDictionaryValuesTotalTime(
-          meta.getPartitionID(), System.currentTimeMillis());
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordCsvInputStepTime(
-          meta.getPartitionID(), System.currentTimeMillis());
-      first = false;
-      data.outputRowMeta = new RowMeta();
-      meta.getFields(data.outputRowMeta, getStepname(), null, null, this);
-
-      // We only run in parallel if we have at least one file to process
-      // AND if we have more than one step copy running...
-      //
-      data.parallel = meta.isRunningInParallel() && data.totalNumberOfSteps > 1;
-
-      // The conversion logic for when the lazy conversion is turned of is simple:
-      // Pretend it's a lazy conversion object anyway and get the native type during
-      // conversion.
-      //
-      data.convertRowMeta = data.outputRowMeta.clone();
-
-      for (ValueMetaInterface valueMeta : data.convertRowMeta.getValueMetaList()) {
-        valueMeta.setStorageType(ValueMetaInterface.STORAGE_TYPE_BINARY_STRING);
-      }
-
-      // Calculate the indexes for the filename and row number fields
-      //
-      data.filenameFieldIndex = -1;
-      if (!Const.isEmpty(meta.getFilenameField()) && meta.isIncludingFilename()) {
-        data.filenameFieldIndex = meta.getInputFields().length;
-      }
-
-      data.rownumFieldIndex = -1;
-      if (!Const.isEmpty(meta.getRowNumField())) {
-        data.rownumFieldIndex = meta.getInputFields().length;
-        if (data.filenameFieldIndex >= 0) {
-          data.rownumFieldIndex++;
-        }
-      }
-      rddIteratorKey = StringUtils.isEmpty(meta.getRddIteratorKey()) ? null : meta
-              .getRddIteratorKey();
-    }
-
-    // start multi-thread to process
-    int numberOfNodes;
-    try {
-      numberOfNodes = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.NUM_CORES_LOADING,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-    } catch (NumberFormatException exc) {
-      numberOfNodes = NUM_CORES_DEFAULT_VAL;
-    }
-    if (rddIteratorKey == null) {
-      BlockDetails[] blocksInfo = GraphGenerator.blockInfo.get(meta.getBlocksID());
-      if (blocksInfo.length == 0) {
-        //if isDirectLoad = true, and partition number > file num
-        //then blocksInfo will get empty in some partition processing, so just return
-        setOutputDone();
-        return false;
-      }
-
-      if (numberOfNodes > blocksInfo.length) {
-        numberOfNodes = blocksInfo.length;
-      }
-
-      //new the empty lists
-      for (int pos = 0; pos < numberOfNodes; pos++) {
-        threadBlockList.add(new ArrayList<BlockDetails>());
-      }
-
-      //block balance to every thread
-      for (int pos = 0; pos < blocksInfo.length; ) {
-        for (int threadNum = 0; threadNum < numberOfNodes; threadNum++) {
-          if (pos < blocksInfo.length) {
-            threadBlockList.get(threadNum).add(blocksInfo[pos++]);
-          }
-        }
-      }
-      LOGGER.info("*****************Started all csv reading***********");
-      startProcess(numberOfNodes);
-      LOGGER.info("*****************Completed all csv reading***********");
-      CarbonTimeStatisticsFactory.getLoadStatisticsInstance().recordCsvInputStepTime(
-              meta.getPartitionID(), System.currentTimeMillis());
-    } else if (rddIteratorKey.startsWith(CarbonCommonConstants.RDDUTIL_UPDATE_KEY)) {
-      scanRddIteratorForUpdate();
-    }
-    else {
-      scanRddIterator(numberOfNodes);
-    }
-    setOutputDone();
-    return false;
-  }
-
-  class RddScanCallable implements Callable<Void> {
-    @Override public Void call() throws Exception {
-      StandardLogService
-          .setThreadName(("PROCESS_DataFrame_PARTITIONS"), Thread.currentThread().getName());
-      try {
-        String[] values = null;
-        boolean hasNext = true;
-        CarbonIterator<String[]> iter;
-        boolean isInitialized = false;
-        while (hasNext) {
-          // Inovke getRddIterator to get a RDD[Row] iterator of a partition.
-          // The RDD comes from the sub-query DataFrame in InsertInto statement.
-          iter = getRddIterator(isInitialized);
-          isInitialized = true;
-          if (iter == null) {
-            hasNext = false;
-          } else {
-            while (iter.hasNext()) {
-              values = iter.next();
-              synchronized (putRowLock) {
-                putRow(data.outputRowMeta, values);
-              }
-            }
-          }
-        }
-      } catch (Exception e) {
-        LOGGER.error(e, "Scan rdd during data load is terminated due to error.");
-        throw e;
-      }
-      return null;
-    }
-  }
-
-  private synchronized CarbonIterator<String[]> getRddIterator(boolean isInitialized) {
-    if (!isInitialized) {
-      rddIterator.initialize();
-    }
-    if (rddIterator.hasNext()) {
-      return rddIterator.next();
-    }
-    return null;
-  }
-
-  private void scanRddIterator(int numberOfNodes) throws RuntimeException {
-    rddIterator = RddInputUtils.getAndRemove(rddIteratorKey);
-    if (rddIterator != null) {
-      exec = Executors.newFixedThreadPool(numberOfNodes);
-      List<Future<Void>> results = new ArrayList<Future<Void>>(numberOfNodes);
-      RddScanCallable[] calls = new RddScanCallable[numberOfNodes];
-      for (int i = 0; i < numberOfNodes; i++) {
-        calls[i] = new RddScanCallable();
-        results.add(exec.submit(calls[i]));
-      }
-      try {
-        for (Future<Void> futrue : results) {
-          futrue.get();
-        }
-      } catch (InterruptedException | ExecutionException e) {
-        LOGGER.error(e, "Thread InterruptedException");
-        throw new RuntimeException("Thread InterruptedException", e);
-      } finally {
-        exec.shutdownNow();
-      }
-    }
-  }
-
-  private void scanRddIteratorForUpdate() throws RuntimeException {
-    Iterator<String[]> iterator = RddInpututilsForUpdate.getAndRemove(rddIteratorKey);
-    if (iterator != null) {
-      try {
-        while (iterator.hasNext()) {
-          putRow(data.outputRowMeta, iterator.next());
-        }
-      } catch (KettleException e) {
-        throw new RuntimeException(e);
-      } catch (Exception e) {
-        LOGGER.error(e, "Scan rdd during data load is terminated due to error.");
-        throw e;
-      }
-    }
-  }
-
-  private void startProcess(final int numberOfNodes) throws RuntimeException {
-    exec = Executors.newFixedThreadPool(numberOfNodes);
-
-    Callable<Void> callable = new Callable<Void>() {
-      @Override public Void call() throws RuntimeException {
-        StandardLogService.setThreadName(("PROCESS_BLOCKS"), Thread.currentThread().getName());
-        try {
-          LOGGER.info("*****************started csv reading by thread***********");
-          doProcessUnivocity();
-          LOGGER.info("*****************Completed csv reading by thread***********");
-        } catch (Throwable e) {
-          LOGGER.error(e, "Thread is terminated due to error");
-          throw new RuntimeException("Thread is terminated due to error : " + e.getMessage());
-        }
-        return null;
-      }
-    };
-    List<Future<Void>> results = new ArrayList<Future<Void>>(10);
-    for (int i = 0; i < numberOfNodes; i++) {
-      results.add(exec.submit(callable));
-    }
-
-    resultArray = results.toArray(new Future[results.size()]);
-    try {
-      for (int j = 0; j < resultArray.length; j++) {
-        resultArray[j].get();
-      }
-    } catch (InterruptedException | ExecutionException e) {
-      throw new RuntimeException("Thread InterruptedException", e);
-    }
-    finally {
-      exec.shutdownNow();
-    }
-  }
-
-  private void doProcessUnivocity() {
-    List<BlockDetails> blocksListForProcess = null;
-    synchronized (getBlockListLock) {
-      //get the blocksList for this thread
-      blocksListForProcess = threadBlockList.get(threadBlockList.size() - 1);
-      threadBlockList.remove(threadBlockList.size() - 1);
-    }
-    long currentTimeMillis = System.currentTimeMillis();
-    UnivocityCsvParser parser = new UnivocityCsvParser(getParserVo(blocksListForProcess));
-    long numberOfRows = 0;
-    int numberOfColumns = meta.getInputFields().length;
-    try {
-      parser.initialize();
-      while (parser.hasMoreRecords()) {
-        String[] next = parser.getNextRecord();
-        if (next.length < numberOfColumns) {
-          String[] temp = new String[numberOfColumns];
-          System.arraycopy(next, 0, temp, 0, next.length);
-          next = temp;
-        }
-        synchronized (putRowLock) {
-          putRow(data.outputRowMeta, next);
-          numberOfRows++;
-        }
-      }
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    } catch (KettleException e) {
-      throw new RuntimeException(e);
-    }
-    LOGGER.info("Total Number of records processed by this thread is: " + numberOfRows);
-    LOGGER.info("Time taken to processed " + numberOfRows + " Number of records: " + (
-        System.currentTimeMillis() - currentTimeMillis));
-  }
-
-  private UnivocityCsvParserVo getParserVo(List<BlockDetails> blocksListForProcess) {
-    UnivocityCsvParserVo csvParserVo = new UnivocityCsvParserVo();
-    csvParserVo.setBlockDetailsList(blocksListForProcess);
-    csvParserVo.setDelimiter(meta.getDelimiter());
-    csvParserVo.setNumberOfColumns(meta.getInputFields().length);
-    csvParserVo.setEscapeCharacter(meta.getEscapeCharacter());
-    csvParserVo.setHeaderPresent(meta.isHeaderPresent());
-    csvParserVo.setQuoteCharacter(meta.getQuoteCharacter());
-    csvParserVo.setCommentCharacter(meta.getCommentCharacter());
-    String maxColumns = meta.getMaxColumns();
-    if (null != maxColumns) {
-      csvParserVo.setMaxColumns(Integer.parseInt(maxColumns));
-    }
-    return csvParserVo;
-  }
-
-  @Override public void dispose(StepMetaInterface smi, StepDataInterface sdi) {
-    try {
-      // Clean the block info in map
-      if (GraphGenerator.blockInfo.get(meta.getBlocksID()) != null) {
-        GraphGenerator.blockInfo.remove(meta.getBlocksID());
-      }
-    } catch (Exception e) {
-      logError("Error closing file channel", e);
-    }
-
-    super.dispose(smi, sdi);
-  }
-
-  public boolean init(StepMetaInterface smi, StepDataInterface sdi) {
-    meta = (CsvInputMeta) smi;
-    data = (CsvInputData) sdi;
-
-    if (super.init(smi, sdi)) {
-      // If the step doesn't have any previous steps, we just get the filename.
-      // Otherwise, we'll grab the list of filenames later...
-      //
-      if (getTransMeta().findNrPrevSteps(getStepMeta()) == 0) {
-        String filename = environmentSubstitute(meta.getFilename());
-
-        if (Const.isEmpty(filename) && Const.isEmpty(meta.getRddIteratorKey())) {
-          logError(BaseMessages.getString(PKG, "CsvInput.MissingFilename.Message")); //$NON-NLS-1$
-          return false;
-        }
-      }
-
-      data.encodingType = EncodingType.guessEncodingType(meta.getEncoding());
-
-      // PDI-2489 - set the delimiter byte value to the code point of the
-      // character as represented in the input file's encoding
-      try {
-        data.delimiter = data.encodingType
-            .getBytes(environmentSubstitute(meta.getDelimiter()), meta.getEncoding());
-        data.escapeCharacter = data.encodingType
-            .getBytes(environmentSubstitute(meta.getEscapeCharacter()), meta.getEncoding());
-        if (Const.isEmpty(meta.getEnclosure())) {
-          data.enclosure = null;
-        } else {
-          data.enclosure = data.encodingType
-              .getBytes(environmentSubstitute(meta.getEnclosure()), meta.getEncoding());
-        }
-
-      } catch (UnsupportedEncodingException e) {
-        logError(BaseMessages.getString(PKG, "CsvInput.BadEncoding.Message"), e); //$NON-NLS-1$
-        return false;
-      }
-
-      // Handle parallel reading capabilities...
-      //
-
-      if (meta.isRunningInParallel()) {
-        data.totalNumberOfSteps = getUniqueStepCountAcrossSlaves();
-
-        // We are not handling a single file, but possibly a list of files...
-        // As such, the fair thing to do is calculate the total size of the files
-        // Then read the required block.
-        //
-
-      }
-      return true;
-    }
-    return false;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputData.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputData.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputData.java
deleted file mode 100644
index 6114eb9..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputData.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.csvreaderstep;
-
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.trans.step.BaseStepData;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.steps.textfileinput.EncodingType;
-
-public class CsvInputData extends BaseStepData implements StepDataInterface {
-  public RowMetaInterface convertRowMeta;
-  public RowMetaInterface outputRowMeta;
-
-  public byte[] delimiter;
-  public byte[] enclosure;
-  public byte[] escapeCharacter;
-  public int preferredBufferSize;
-  public int totalNumberOfSteps;
-  public boolean parallel;
-  public int filenameFieldIndex;
-  public int rownumFieldIndex;
-  /**
-   * <pre>
-   * if true then when double enclosure appears one will be considered as escape enclosure
-   * Ecample: 'abc''xyz' would be processed as abc'xyz
-   * </pre>
-   */
-  public EncodingType encodingType;
-  public CsvInputData() {
-    super();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java
deleted file mode 100644
index d0ba585..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java
+++ /dev/null
@@ -1,971 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.carbondata.processing.csvreaderstep;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.vfs.FileObject;
-import org.pentaho.di.core.CheckResult;
-import org.pentaho.di.core.CheckResultInterface;
-import org.pentaho.di.core.Const;
-import org.pentaho.di.core.Counter;
-import org.pentaho.di.core.KettleAttributeInterface;
-import org.pentaho.di.core.database.DatabaseMeta;
-import org.pentaho.di.core.exception.KettleException;
-import org.pentaho.di.core.exception.KettleStepException;
-import org.pentaho.di.core.exception.KettleXMLException;
-import org.pentaho.di.core.row.RowMetaInterface;
-import org.pentaho.di.core.row.ValueMeta;
-import org.pentaho.di.core.row.ValueMetaInterface;
-import org.pentaho.di.core.variables.VariableSpace;
-import org.pentaho.di.core.vfs.KettleVFS;
-import org.pentaho.di.core.xml.XMLHandler;
-import org.pentaho.di.i18n.BaseMessages;
-import org.pentaho.di.repository.ObjectId;
-import org.pentaho.di.repository.Repository;
-import org.pentaho.di.resource.ResourceDefinition;
-import org.pentaho.di.resource.ResourceEntry;
-import org.pentaho.di.resource.ResourceEntry.ResourceType;
-import org.pentaho.di.resource.ResourceNamingInterface;
-import org.pentaho.di.resource.ResourceReference;
-import org.pentaho.di.trans.Trans;
-import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.BaseStepMeta;
-import org.pentaho.di.trans.step.StepDataInterface;
-import org.pentaho.di.trans.step.StepInjectionMetaEntry;
-import org.pentaho.di.trans.step.StepInterface;
-import org.pentaho.di.trans.step.StepMeta;
-import org.pentaho.di.trans.step.StepMetaInjectionInterface;
-import org.pentaho.di.trans.step.StepMetaInterface;
-import org.pentaho.di.trans.steps.textfileinput.InputFileMetaInterface;
-import org.pentaho.di.trans.steps.textfileinput.TextFileInputField;
-import org.pentaho.di.trans.steps.textfileinput.TextFileInputMeta;
-import org.w3c.dom.Node;
-
-public class CsvInputMeta extends BaseStepMeta
-    implements StepMetaInterface, InputFileMetaInterface, StepMetaInjectionInterface {
-  private static final Class<?> PKG = CsvInput.class;
-
-  private String filename;
-
-  private String filenameField;
-
-  private boolean includingFilename;
-
-  private String rowNumField;
-
-  private boolean headerPresent;
-
-  private String delimiter;
-  private String enclosure;
-  private boolean escapeEnclosure;
-
-  private String bufferSize;
-
-  private boolean lazyConversionActive;
-
-  private TextFileInputField[] inputFields;
-
-  private boolean isaddresult;
-
-  private boolean runningInParallel;
-
-  private String encoding;
-
-  private boolean newlinePossibleInFields;
-
-  private String blocksID;
-
-  private String partitionID;
-
-  private String escapeCharacter;
-
-  private String quoteCharacter;
-
-  private String commentCharacter;
-
-  private String rddIteratorKey;
-
-  private String maxColumns;
-
-  public CsvInputMeta() {
-    super(); // allocate BaseStepMeta
-    allocate(0);
-  }
-
-  public void loadXML(Node stepnode, List<DatabaseMeta> databases, Map<String, Counter> counters)
-      throws KettleXMLException {
-    readData(stepnode);
-  }
-
-  public void setDefault() {
-    delimiter = ",";
-    enclosure = "\"";
-    headerPresent = true;
-    lazyConversionActive = true;
-    isaddresult = false;
-    bufferSize = "50000";
-    blocksID = "";
-    partitionID = "";
-    escapeCharacter = "\\";
-    quoteCharacter = "\"";
-    commentCharacter = "#";
-    rddIteratorKey = "";
-  }
-
-  private void readData(Node stepnode) throws KettleXMLException {
-    try {
-      filename = XMLHandler.getTagValue(stepnode, getXmlCode("FILENAME"));
-      filenameField = XMLHandler.getTagValue(stepnode, getXmlCode("FILENAME_FIELD"));
-      rowNumField = XMLHandler.getTagValue(stepnode, getXmlCode("ROW_NUM_FIELD"));
-      includingFilename =
-          "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("INCLUDE_FILENAME")));
-      delimiter = XMLHandler.getTagValue(stepnode, getXmlCode("DELIMITER"));
-      enclosure = XMLHandler.getTagValue(stepnode, getXmlCode("ENCLOSURE"));
-      escapeEnclosure =
-          "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("ESACAPE_ENCLOSURE")));
-      bufferSize = XMLHandler.getTagValue(stepnode, getXmlCode("BUFFERSIZE"));
-      headerPresent =
-          "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("HEADER_PRESENT")));
-      lazyConversionActive =
-          "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("LAZY_CONVERSION")));
-      isaddresult =
-          "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("ADD_FILENAME_RESULT")));
-      runningInParallel =
-          "Y".equalsIgnoreCase(XMLHandler.getTagValue(stepnode, getXmlCode("PARALLEL")));
-      String nlp = XMLHandler.getTagValue(stepnode, getXmlCode("NEWLINE_POSSIBLE"));
-      if (Const.isEmpty(nlp)) {
-        if (runningInParallel) {
-          newlinePossibleInFields = false;
-        } else {
-          newlinePossibleInFields = true;
-        }
-      } else {
-        newlinePossibleInFields = "Y".equalsIgnoreCase(nlp);
-      }
-      encoding = XMLHandler.getTagValue(stepnode, getXmlCode("ENCODING"));
-      blocksID = XMLHandler.getTagValue(stepnode, "blocksID");
-      partitionID = XMLHandler.getTagValue(stepnode, "partitionID");
-      escapeCharacter = XMLHandler.getTagValue(stepnode, "escapeCharacter");
-      quoteCharacter = XMLHandler.getTagValue(stepnode, "quoteCharacter");
-      commentCharacter = XMLHandler.getTagValue(stepnode, "commentCharacter");
-      rddIteratorKey = XMLHandler.getTagValue(stepnode, "rddIteratorKey");
-      maxColumns = XMLHandler.getTagValue(stepnode, "maxColumns");
-      Node fields = XMLHandler.getSubNode(stepnode, getXmlCode("FIELDS"));
-      int nrfields = XMLHandler.countNodes(fields, getXmlCode("FIELD"));
-
-      allocate(nrfields);
-
-      for (int i = 0; i < nrfields; i++) {
-        inputFields[i] = new TextFileInputField();
-
-        Node fnode = XMLHandler.getSubNodeByNr(fields, getXmlCode("FIELD"), i);
-
-        inputFields[i].setName(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_NAME")));
-        inputFields[i]
-            .setType(ValueMeta.getType(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_TYPE"))));
-        inputFields[i].setFormat(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_FORMAT")));
-        inputFields[i]
-            .setCurrencySymbol(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_CURRENCY")));
-        inputFields[i].setDecimalSymbol(XMLHandler.getTagValue(fnode,
-            getXmlCode("FIELD_DECIMAL")));
-        inputFields[i].setGroupSymbol(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_GROUP")));
-        inputFields[i]
-            .setLength(Const.toInt(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_LENGTH")), -1));
-        inputFields[i].setPrecision(
-            Const.toInt(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_PRECISION")), -1));
-        inputFields[i].setTrimType(ValueMeta
-            .getTrimTypeByCode(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_TRIM_TYPE"))));
-      }
-    } catch (Exception e) {
-      throw new KettleXMLException("Unable to load step info from XML", e);
-    }
-  }
-
-  public void allocate(int nrFields) {
-    inputFields = new TextFileInputField[nrFields];
-  }
-
-  public String getXML() {
-    StringBuilder retval = new StringBuilder(500);
-
-    retval.append("    ").append(XMLHandler.addTagValue(getXmlCode("FILENAME"), filename));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue(getXmlCode("FILENAME_FIELD"), filenameField));
-    retval.append("    ").append(XMLHandler.addTagValue(getXmlCode("ROW_NUM_FIELD"), rowNumField));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue(getXmlCode("INCLUDE_FILENAME"), includingFilename));
-    retval.append("    ").append(XMLHandler.addTagValue(getXmlCode("DELIMITER"), delimiter));
-    retval.append("    ").append(XMLHandler.addTagValue(getXmlCode("ENCLOSURE"), enclosure));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue(getXmlCode("ESACAPE_ENCLOSURE"), escapeEnclosure));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue(getXmlCode("HEADER_PRESENT"), headerPresent));
-    retval.append("    ").append(XMLHandler.addTagValue(getXmlCode("BUFFERSIZE"), bufferSize));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue(getXmlCode("LAZY_CONVERSION"), lazyConversionActive));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue(getXmlCode("ADD_FILENAME_RESULT"), isaddresult));
-    retval.append("    ").append(XMLHandler.addTagValue(getXmlCode("PARALLEL"), runningInParallel));
-    retval.append("    ")
-        .append(XMLHandler.addTagValue(getXmlCode("NEWLINE_POSSIBLE"), newlinePossibleInFields));
-    retval.append("    ").append(XMLHandler.addTagValue(getXmlCode("ENCODING"), encoding));
-    retval.append("    ").append(XMLHandler.addTagValue("blocksID", blocksID));
-    retval.append("    ").append(XMLHandler.addTagValue("partitionID", partitionID));
-    retval.append("    ").append(XMLHandler.addTagValue("escapeCharacter", escapeCharacter));
-    retval.append("    ").append(XMLHandler.addTagValue("quoteCharacter", quoteCharacter));
-    retval.append("    ").append(XMLHandler.addTagValue("commentCharacter", commentCharacter));
-    retval.append("    ").append(XMLHandler.addTagValue("rddIteratorKey", rddIteratorKey));
-    retval.append("    ").append(XMLHandler.addTagValue("maxColumns", maxColumns));
-    retval.append("    ").append(XMLHandler.openTag(getXmlCode("FIELDS"))).append(Const.CR);
-    for (int i = 0; i < inputFields.length; i++) {
-      TextFileInputField field = inputFields[i];
-
-      retval.append("      ").append(XMLHandler.openTag(getXmlCode("FIELD"))).append(Const.CR);
-      retval.append("        ")
-          .append(XMLHandler.addTagValue(getXmlCode("FIELD_NAME"), field.getName()));
-      retval.append("        ").append(
-          XMLHandler.addTagValue(getXmlCode("FIELD_TYPE"), ValueMeta.getTypeDesc(field.getType())));
-      retval.append("        ")
-          .append(XMLHandler.addTagValue(getXmlCode("FIELD_FORMAT"), field.getFormat()));
-      retval.append("        ")
-          .append(XMLHandler.addTagValue(getXmlCode("FIELD_CURRENCY"), field.getCurrencySymbol()));
-      retval.append("        ")
-          .append(XMLHandler.addTagValue(getXmlCode("FIELD_DECIMAL"), field.getDecimalSymbol()));
-      retval.append("        ")
-          .append(XMLHandler.addTagValue(getXmlCode("FIELD_GROUP"), field.getGroupSymbol()));
-      retval.append("        ")
-          .append(XMLHandler.addTagValue(getXmlCode("FIELD_LENGTH"), field.getLength()));
-      retval.append("        ")
-          .append(XMLHandler.addTagValue(getXmlCode("FIELD_PRECISION"), field.getPrecision()));
-      retval.append("        ").append(XMLHandler.addTagValue(getXmlCode("FIELD_TRIM_TYPE"),
-          ValueMeta.getTrimTypeCode(field.getTrimType())));
-      retval.append("      ").append(XMLHandler.closeTag(getXmlCode("FIELD"))).append(Const.CR);
-    }
-    retval.append("    ").append(XMLHandler.closeTag(getXmlCode("FIELDS"))).append(Const.CR);
-
-    return retval.toString();
-  }
-
-  public void readRep(Repository rep, ObjectId idStep, List<DatabaseMeta> databases,
-      Map<String, Counter> counters) throws KettleException {
-    try {
-      filename = rep.getStepAttributeString(idStep, getRepCode("FILENAME"));
-      filenameField = rep.getStepAttributeString(idStep, getRepCode("FILENAME_FIELD"));
-      rowNumField = rep.getStepAttributeString(idStep, getRepCode("ROW_NUM_FIELD"));
-      includingFilename = rep.getStepAttributeBoolean(idStep, getRepCode("INCLUDE_FILENAME"));
-      delimiter = rep.getStepAttributeString(idStep, getRepCode("DELIMITER"));
-      enclosure = rep.getStepAttributeString(idStep, getRepCode("ENCLOSURE"));
-      escapeEnclosure = rep.getStepAttributeBoolean(idStep, getRepCode("ESACAPE_ENCLOSURE"));
-      headerPresent = rep.getStepAttributeBoolean(idStep, getRepCode("HEADER_PRESENT"));
-      bufferSize = rep.getStepAttributeString(idStep, getRepCode("BUFFERSIZE"));
-      lazyConversionActive = rep.getStepAttributeBoolean(idStep, getRepCode("LAZY_CONVERSION"));
-      isaddresult = rep.getStepAttributeBoolean(idStep, getRepCode("ADD_FILENAME_RESULT"));
-      runningInParallel = rep.getStepAttributeBoolean(idStep, getRepCode("PARALLEL"));
-      newlinePossibleInFields =
-          rep.getStepAttributeBoolean(idStep, 0, getRepCode("NEWLINE_POSSIBLE"),
-              !runningInParallel);
-      encoding = rep.getStepAttributeString(idStep, getRepCode("ENCODING"));
-      blocksID = rep.getStepAttributeString(idStep, getRepCode("blocksID"));
-      partitionID = rep.getStepAttributeString(idStep, getRepCode("partitionID"));
-      escapeCharacter = rep.getStepAttributeString(idStep, getRepCode("escapeCharacter"));
-      quoteCharacter = rep.getStepAttributeString(idStep, getRepCode("quoteCharacter"));
-      commentCharacter = rep.getStepAttributeString(idStep, getRepCode("commentCharacter"));
-      rddIteratorKey = rep.getStepAttributeString(idStep, getRepCode("rddIteratorKey"));
-      maxColumns = rep.getStepAttributeString(idStep, getRepCode("maxColumns"));
-      int nrfields = rep.countNrStepAttributes(idStep, getRepCode("FIELD_NAME"));
-
-      allocate(nrfields);
-
-      for (int i = 0; i < nrfields; i++) {
-        inputFields[i] = new TextFileInputField();
-
-        inputFields[i].setName(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_NAME")));
-        inputFields[i].setType(
-            ValueMeta.getType(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_TYPE"))));
-        inputFields[i].setFormat(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_FORMAT")));
-        inputFields[i]
-            .setCurrencySymbol(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_CURRENCY")));
-        inputFields[i]
-            .setDecimalSymbol(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_DECIMAL")));
-        inputFields[i]
-            .setGroupSymbol(rep.getStepAttributeString(idStep, i, getRepCode("FIELD_GROUP")));
-        inputFields[i]
-            .setLength((int) rep.getStepAttributeInteger(idStep, i, getRepCode("FIELD_LENGTH")));
-        inputFields[i].setPrecision(
-            (int) rep.getStepAttributeInteger(idStep, i, getRepCode("FIELD_PRECISION")));
-        inputFields[i].setTrimType(ValueMeta.getTrimTypeByCode(
-            rep.getStepAttributeString(idStep, i, getRepCode("FIELD_TRIM_TYPE"))));
-      }
-    } catch (Exception e) {
-      throw new KettleException("Unexpected error reading step information from the repository", e);
-    }
-  }
-
-  public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep)
-      throws KettleException {
-    try {
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("FILENAME"), filename);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("FILENAME_FIELD"), filenameField);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("ROW_NUM_FIELD"), rowNumField);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("INCLUDE_FILENAME"),
-          includingFilename);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("DELIMITER"), delimiter);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("ENCLOSURE"), enclosure);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("ESACAPE_ENCLOSURE"),
-          escapeEnclosure);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("BUFFERSIZE"), bufferSize);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("HEADER_PRESENT"), headerPresent);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("LAZY_CONVERSION"),
-          lazyConversionActive);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("ADD_FILENAME_RESULT"),
-          isaddresult);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("PARALLEL"), runningInParallel);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("NEWLINE_POSSIBLE"),
-          newlinePossibleInFields);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("ENCODING"), encoding);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("blocksID"), blocksID);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("partitionID"), partitionID);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("escapeCharacter"),
-          escapeCharacter);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("quoteCharacter"),
-          quoteCharacter);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("commentCharacter"),
-          commentCharacter);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("rddIteratorKey"),
-              rddIteratorKey);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("maxColumns"),
-          maxColumns);
-      for (int i = 0; i < inputFields.length; i++) {
-        TextFileInputField field = inputFields[i];
-
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_NAME"),
-            field.getName());
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_TYPE"),
-            ValueMeta.getTypeDesc(field.getType()));
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_FORMAT"),
-            field.getFormat());
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_CURRENCY"),
-            field.getCurrencySymbol());
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_DECIMAL"),
-            field.getDecimalSymbol());
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_GROUP"),
-            field.getGroupSymbol());
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_LENGTH"),
-            field.getLength());
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_PRECISION"),
-            field.getPrecision());
-        rep.saveStepAttribute(idTransformation, idStep, i, getRepCode("FIELD_TRIM_TYPE"),
-            ValueMeta.getTrimTypeCode(field.getTrimType()));
-      }
-    } catch (Exception e) {
-      throw new KettleException(
-          "Unable to save step information to the repository for id_step=" + idStep, e);
-    }
-  }
-
-  public void getFields(RowMetaInterface rowMeta, String origin, RowMetaInterface[] info,
-      StepMeta nextStep, VariableSpace space) throws KettleStepException {
-    rowMeta.clear(); // Start with a clean slate, eats the input
-
-    for (int i = 0; i < inputFields.length; i++) {
-      TextFileInputField field = inputFields[i];
-
-      ValueMetaInterface valueMeta = new ValueMeta(field.getName(), field.getType());
-      valueMeta.setConversionMask(field.getFormat());
-      valueMeta.setLength(field.getLength());
-      valueMeta.setPrecision(field.getPrecision());
-      valueMeta.setConversionMask(field.getFormat());
-      valueMeta.setDecimalSymbol(field.getDecimalSymbol());
-      valueMeta.setGroupingSymbol(field.getGroupSymbol());
-      valueMeta.setCurrencySymbol(field.getCurrencySymbol());
-      valueMeta.setTrimType(field.getTrimType());
-      if (lazyConversionActive) {
-        valueMeta.setStorageType(ValueMetaInterface.STORAGE_TYPE_BINARY_STRING);
-      }
-      valueMeta.setStringEncoding(space.environmentSubstitute(encoding));
-
-      // In case we want to convert Strings...
-      // Using a copy of the valueMeta object means that the inner and outer representation
-      // format is the same.
-      // Preview will show the data the same way as we read it.
-      // This layout is then taken further down the road by the metadata through the transformation.
-      //
-      ValueMetaInterface storageMetadata = valueMeta.clone();
-      storageMetadata.setType(ValueMetaInterface.TYPE_STRING);
-      storageMetadata.setStorageType(ValueMetaInterface.STORAGE_TYPE_NORMAL);
-      storageMetadata
-          .setLength(-1, -1); // we don't really know the lengths of the strings read in advance.
-      valueMeta.setStorageMetadata(storageMetadata);
-
-      valueMeta.setOrigin(origin);
-
-      rowMeta.addValueMeta(valueMeta);
-    }
-
-    if (!Const.isEmpty(filenameField) && includingFilename) {
-      ValueMetaInterface filenameMeta =
-          new ValueMeta(filenameField, ValueMetaInterface.TYPE_STRING);
-      filenameMeta.setOrigin(origin);
-      if (lazyConversionActive) {
-        filenameMeta.setStorageType(ValueMetaInterface.STORAGE_TYPE_BINARY_STRING);
-        filenameMeta
-            .setStorageMetadata(new ValueMeta(filenameField, ValueMetaInterface.TYPE_STRING));
-      }
-      rowMeta.addValueMeta(filenameMeta);
-    }
-
-    if (!Const.isEmpty(rowNumField)) {
-      ValueMetaInterface rowNumMeta = new ValueMeta(rowNumField, ValueMetaInterface.TYPE_INTEGER);
-      rowNumMeta.setLength(10);
-      rowNumMeta.setOrigin(origin);
-      rowMeta.addValueMeta(rowNumMeta);
-    }
-
-  }
-
-  public void check(List<CheckResultInterface> remarks, TransMeta transMeta, StepMeta stepinfo,
-      RowMetaInterface prev, String[] input, String[] output, RowMetaInterface info) {
-    CheckResult cr;
-    if (prev == null || prev.size() == 0) {
-      cr = new CheckResult(CheckResultInterface.TYPE_RESULT_OK,
-          BaseMessages.getString(PKG, "CsvInputMeta.CheckResult.NotReceivingFields"),
-          stepinfo); //$NON-NLS-1$
-      remarks.add(cr);
-    } else {
-      cr = new CheckResult(CheckResultInterface.TYPE_RESULT_ERROR, BaseMessages
-          .getString(PKG, "CsvInputMeta.CheckResult.StepRecevingData", prev.size() + ""),
-          stepinfo); //$NON-NLS-1$ //$NON-NLS-2$
-      remarks.add(cr);
-    }
-
-    // See if we have input streams leading to this step!
-    if (input.length > 0) {
-      cr = new CheckResult(CheckResultInterface.TYPE_RESULT_ERROR,
-          BaseMessages.getString(PKG, "CsvInputMeta.CheckResult.StepRecevingData2"),
-          stepinfo); //$NON-NLS-1$
-      remarks.add(cr);
-    } else {
-      cr = new CheckResult(CheckResultInterface.TYPE_RESULT_OK,
-          BaseMessages.getString(PKG, "CsvInputMeta.CheckResult.NoInputReceivedFromOtherSteps"),
-          stepinfo); //$NON-NLS-1$
-      remarks.add(cr);
-    }
-  }
-
-  public StepInterface getStep(StepMeta stepMeta, StepDataInterface stepDataInterface, int cnr,
-      TransMeta tr, Trans trans) {
-    return new CsvInput(stepMeta, stepDataInterface, cnr, tr, trans);
-  }
-
-  public StepDataInterface getStepData() {
-    return new CsvInputData();
-  }
-
-  /**
-   * @return the delimiter
-   */
-  public String getDelimiter() {
-    return delimiter;
-  }
-
-  /**
-   * @param delimiter the delimiter to set
-   */
-  public void setDelimiter(String delimiter) {
-    this.delimiter = delimiter;
-  }
-
-  /**
-   * @return the filename
-   */
-  public String getFilename() {
-    return filename;
-  }
-
-  /**
-   * @param filename the filename to set
-   */
-  public void setFilename(String filename) {
-    this.filename = filename;
-  }
-
-  /**
-   * @return the bufferSize
-   */
-  public String getBufferSize() {
-    return bufferSize;
-  }
-
-  /**
-   * @param bufferSize the bufferSize to set
-   */
-  public void setBufferSize(String bufferSize) {
-    this.bufferSize = bufferSize;
-  }
-
-  /**
-   * @return true if lazy conversion is turned on: conversions are delayed as long as possible,
-   * perhaps to never occur at all.
-   */
-  public boolean isLazyConversionActive() {
-    return lazyConversionActive;
-  }
-
-  /**
-   * @param lazyConversionActive true if lazy conversion is to be turned on: conversions are
-   *                             delayed as long as possible, perhaps to never occur at all.
-   */
-  public void setLazyConversionActive(boolean lazyConversionActive) {
-    this.lazyConversionActive = lazyConversionActive;
-  }
-
-  /**
-   * @return the headerPresent
-   */
-  public boolean isHeaderPresent() {
-    return headerPresent;
-  }
-
-  /**
-   * @param headerPresent the headerPresent to set
-   */
-  public void setHeaderPresent(boolean headerPresent) {
-    this.headerPresent = headerPresent;
-  }
-
-  /**
-   * @return the enclosure
-   */
-  public String getEnclosure() {
-    return enclosure;
-  }
-
-  /**
-   * @param enclosure the enclosure to set
-   */
-  public void setEnclosure(String enclosure) {
-    this.enclosure = enclosure;
-  }
-
-  /**
-   * @return the escapeEnclosure
-   */
-  public boolean isEscapeEnclosure() {
-    return escapeEnclosure;
-  }
-
-  /**
-   * @param escapeEnclosure the escapeEnclosure to set
-   */
-  public void setEscapeEnclosure(boolean escapeEnclosure) {
-    this.escapeEnclosure = escapeEnclosure;
-  }
-
-  @Override
-  public List<ResourceReference> getResourceDependencies(TransMeta transMeta, StepMeta stepInfo) {
-    List<ResourceReference> references = new ArrayList<ResourceReference>(5);
-
-    ResourceReference reference = new ResourceReference(stepInfo);
-    references.add(reference);
-    if (!Const.isEmpty(filename)) {
-      // Add the filename to the references, including a reference to this
-      // step meta data.
-      //
-      reference.getEntries()
-          .add(new ResourceEntry(transMeta.environmentSubstitute(filename), ResourceType.FILE));
-    }
-    return references;
-  }
-
-  /**
-   * @return the inputFields
-   */
-  public TextFileInputField[] getInputFields() {
-    return inputFields;
-  }
-
-  /**
-   * @param inputFields the inputFields to set
-   */
-  public void setInputFields(TextFileInputField[] inputFields) {
-    this.inputFields = inputFields;
-  }
-
-  public int getFileFormatTypeNr() {
-    return TextFileInputMeta.FILE_FORMAT_MIXED;
-  }
-
-  public String[] getFilePaths(VariableSpace space) {
-    return new String[] { space.environmentSubstitute(filename), };
-  }
-
-  public int getNrHeaderLines() {
-    return 1;
-  }
-
-  public boolean hasHeader() {
-    return isHeaderPresent();
-  }
-
-  public String getErrorCountField() {
-    return null;
-  }
-
-  public String getErrorFieldsField() {
-    return null;
-  }
-
-  public String getErrorTextField() {
-    return null;
-  }
-
-  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 getFileType() {
-    return "CSV";
-  }
-
-  public String getSeparator() {
-    return delimiter;
-  }
-
-  public boolean includeFilename() {
-    return false;
-  }
-
-  public boolean includeRowNumber() {
-    return false;
-  }
-
-  public boolean isErrorIgnored() {
-    return false;
-  }
-
-  public boolean isErrorLineSkipped() {
-    return false;
-  }
-
-  /**
-   * @return the filenameField
-   */
-  public String getFilenameField() {
-    return filenameField;
-  }
-
-  /**
-   * @param filenameField the filenameField to set
-   */
-  public void setFilenameField(String filenameField) {
-    this.filenameField = filenameField;
-  }
-
-  /**
-   * @return the includingFilename
-   */
-  public boolean isIncludingFilename() {
-    return includingFilename;
-  }
-
-  /**
-   * @param includingFilename the includingFilename to set
-   */
-  public void setIncludingFilename(boolean includingFilename) {
-    this.includingFilename = includingFilename;
-  }
-
-  /**
-   * @return the rowNumField
-   */
-  public String getRowNumField() {
-    return rowNumField;
-  }
-
-  /**
-   * @param rowNumField the rowNumField to set
-   */
-  public void setRowNumField(String rowNumField) {
-    this.rowNumField = rowNumField;
-  }
-
-  /**
-   * @return Returns isaddresult.
-   */
-  public boolean isAddResultFile() {
-    return isaddresult;
-  }
-
-  /**
-   * @param isaddresult The isaddresult to set.
-   */
-  public void setAddResultFile(boolean isaddresult) {
-    this.isaddresult = isaddresult;
-  }
-
-  /**
-   * @return the runningInParallel
-   */
-  public boolean isRunningInParallel() {
-    return runningInParallel;
-  }
-
-  /**
-   * @param runningInParallel the runningInParallel to set
-   */
-  public void setRunningInParallel(boolean runningInParallel) {
-    this.runningInParallel = runningInParallel;
-  }
-
-  /**
-   * @return the encoding
-   */
-  public String getEncoding() {
-    return encoding;
-  }
-
-  /**
-   * @param encoding the encoding to set
-   */
-  public void setEncoding(String encoding) {
-    this.encoding = encoding;
-  }
-
-  public String getBlocksID() {
-    return blocksID;
-  }
-
-  public void setBlocksID(String blocksID) {
-    this.blocksID = blocksID;
-  }
-
-  /**
-   * Since the exported transformation that runs this will reside in a ZIP file, we can't
-   * reference files relatively.
-   * So what this does is turn the name of files into absolute paths OR it simply includes the
-   * resource in the ZIP file.
-   * For now, we'll simply turn it into an absolute path and pray that the file is on a shared
-   * drive or something like that.
-   * HANDLER: create options to configure this behavior
-   */
-  public String exportResources(VariableSpace space, Map<String, ResourceDefinition> definitions,
-      ResourceNamingInterface resourceNamingInterface, Repository repository)
-      throws KettleException {
-    try {
-      // The object that we're modifying here is a copy of the original!
-      // So let's change the filename from relative to absolute by grabbing the file object...
-      // In case the name of the file comes from previous steps, forget about this!
-      //
-      if (Const.isEmpty(filenameField)) {
-        // From : ${Internal.Transformation.Filename.Directory}/../foo/bar.csv
-        // To   : /home/matt/test/files/foo/bar.csv
-        //
-        FileObject fileObject =
-            KettleVFS.getFileObject(space.environmentSubstitute(filename), space);
-
-        // If the file doesn't exist, forget about this effort too!
-        //
-        if (fileObject.exists()) {
-          // Convert to an absolute path...
-          //
-          filename = resourceNamingInterface.nameResource(fileObject, space, true);
-
-          return filename;
-        }
-      }
-      return null;
-    } catch (Exception e) {
-      throw new KettleException(e); //$NON-NLS-1$
-    }
-  }
-
-  public boolean supportsErrorHandling() {
-    return true;
-  }
-
-  public StepMetaInjectionInterface getStepMetaInjectionInterface() {
-
-    return this;
-  }
-
-  public void injectStepMetadataEntries(List<StepInjectionMetaEntry> metadata) {
-    for (StepInjectionMetaEntry entry : metadata) {
-      KettleAttributeInterface attr = findAttribute(entry.getKey());
-
-      // Set top level attributes...
-      //
-      String attributeKey = attr.getKey();
-      if (entry.getValueType() != ValueMetaInterface.TYPE_NONE) {
-        if ("FILENAME".equals(attributeKey)) {
-          filename = (String) entry.getValue();
-        } else if ("FILENAME_FIELD".equals(attributeKey)) {
-          filenameField = (String) entry.getValue();
-        } else if ("ROW_NUM_FIELD".equals(attributeKey)) {
-          rowNumField = (String) entry.getValue();
-        } else if ("HEADER_PRESENT".equals(attributeKey)) {
-          headerPresent = (Boolean) entry.getValue();
-        } else if ("DELIMITER".equals(attributeKey)) {
-          delimiter = (String) entry.getValue();
-        } else if ("ENCLOSURE".equals(attributeKey)) {
-          enclosure = (String) entry.getValue();
-        } else if ("ESACAPE_ENCLOSURE".equals(attributeKey)) {
-          escapeEnclosure = (Boolean) entry.getValue();
-        } else if ("BUFFERSIZE".equals(attributeKey)) {
-          bufferSize = (String) entry.getValue();
-        } else if ("LAZY_CONVERSION".equals(attributeKey)) {
-          lazyConversionActive = (Boolean) entry.getValue();
-        } else if ("PARALLEL".equals(attributeKey)) {
-          runningInParallel = (Boolean) entry.getValue();
-        } else if ("NEWLINE_POSSIBLE".equals(attributeKey)) {
-          newlinePossibleInFields = (Boolean) entry.getValue();
-        } else if ("ADD_FILENAME_RESULT".equals(attributeKey)) {
-          isaddresult = (Boolean) entry.getValue();
-        } else if ("ENCODING".equals(attributeKey)) {
-          encoding = (String) entry.getValue();
-        } else if ("blocksID".equals(attributeKey)) {
-          blocksID = (String) entry.getValue();
-        } else if ("partitionID".equals(attributeKey)) {
-          partitionID = (String) entry.getValue();
-        } else if ("escapeCharacter".equals(attributeKey)) {
-          escapeCharacter = (String) entry.getValue();
-        } else if ("quoteCharacter".equals(attributeKey)) {
-          quoteCharacter = (String) entry.getValue();
-        } else if ("commentCharacter".equals(attributeKey)) {
-          commentCharacter = (String) entry.getValue();
-        } else if ("rddIteratorKey".equals(attributeKey)) {
-          rddIteratorKey = (String) entry.getValue();
-        } else {
-          throw new RuntimeException(
-              "Unhandled metadata injection of attribute: " + attr.toString() + " - " + attr
-                  .getDescription());
-        }
-      } else {
-        if ("FIELDS".equals(attributeKey)) {
-          // This entry contains a list of lists...
-          // Each list contains a single CSV input field definition (one line in the dialog)
-          //
-          List<StepInjectionMetaEntry> inputFieldEntries = entry.getDetails();
-          inputFields = new TextFileInputField[inputFieldEntries.size()];
-          for (int row = 0; row < inputFieldEntries.size(); row++) {
-            StepInjectionMetaEntry inputFieldEntry = inputFieldEntries.get(row);
-            TextFileInputField inputField = new TextFileInputField();
-
-            List<StepInjectionMetaEntry> fieldAttributes = inputFieldEntry.getDetails();
-            for (int i = 0; i < fieldAttributes.size(); i++) {
-              StepInjectionMetaEntry fieldAttribute = fieldAttributes.get(i);
-              KettleAttributeInterface fieldAttr = findAttribute(fieldAttribute.getKey());
-
-              String attributeValue = (String) fieldAttribute.getValue();
-              String fieldAttributeKey = fieldAttr.getKey();
-              if ("FIELD_NAME".equals(fieldAttributeKey)) {
-                inputField.setName(attributeValue);
-              } else if ("FIELD_TYPE".equals(fieldAttributeKey)) {
-                inputField.setType(ValueMeta.getType(attributeValue));
-              } else if ("FIELD_FORMAT".equals(fieldAttributeKey)) {
-                inputField.setFormat(attributeValue);
-              } else if ("FIELD_LENGTH".equals(fieldAttributeKey)) {
-                inputField
-                    .setLength(attributeValue == null ? -1 : Integer.parseInt(attributeValue));
-              } else if ("FIELD_PRECISION".equals(fieldAttributeKey)) {
-                inputField
-                    .setPrecision(attributeValue == null ? -1 : Integer.parseInt(attributeValue));
-              } else if ("FIELD_CURRENCY".equals(fieldAttributeKey)) {
-                inputField.setCurrencySymbol(attributeValue);
-              } else if ("FIELD_DECIMAL".equals(fieldAttributeKey)) {
-                inputField.setDecimalSymbol(attributeValue);
-              } else if ("FIELD_GROUP".equals(fieldAttributeKey)) {
-                inputField.setGroupSymbol(attributeValue);
-              } else if ("FIELD_TRIM_TYPE".equals(fieldAttributeKey)) {
-                inputField.setTrimType(ValueMeta.getTrimTypeByCode(attributeValue));
-              } else {
-                throw new RuntimeException(
-                    "Unhandled metadata injection of attribute: " + fieldAttr.toString() + " - "
-                        + fieldAttr.getDescription());
-              }
-            }
-
-            inputFields[row] = inputField;
-          }
-        }
-      }
-    }
-  }
-
-  /**
-   * Describe the metadata attributes that can be injected into this step metadata object.
-   *
-   * @throws KettleException
-   */
-  public List<StepInjectionMetaEntry> getStepInjectionMetadataEntries() throws KettleException {
-    return getStepInjectionMetadataEntries(PKG);
-  }
-
-  /**
-   * @return the newlinePossibleInFields
-   */
-  public boolean isNewlinePossibleInFields() {
-    return newlinePossibleInFields;
-  }
-
-  /**
-   * @param newlinePossibleInFields the newlinePossibleInFields to set
-   */
-  public void setNewlinePossibleInFields(boolean newlinePossibleInFields) {
-    this.newlinePossibleInFields = newlinePossibleInFields;
-  }
-
-  public void setPartitionID(String partitionID) {
-    this.partitionID = partitionID;
-  }
-
-  public String getPartitionID() {
-    return this.partitionID;
-  }
-
-  public String getMaxColumns() {
-    return maxColumns;
-  }
-
-  public void setMaxColumns(String maxColumns) {
-    this.maxColumns = maxColumns;
-  }
-
-  public String getRddIteratorKey() {
-    return this.rddIteratorKey;
-  }
-  public void setRddIteratorKey(String rddIteratorKey) {
-    this.rddIteratorKey = rddIteratorKey;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInputUtils.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInputUtils.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInputUtils.java
deleted file mode 100644
index 0f5add2..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInputUtils.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.csvreaderstep;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.carbondata.common.CarbonIterator;
-
-public class RddInputUtils {
-  private static Map<String, CarbonIterator<CarbonIterator<String[]>>> iteratorMap = new
-      HashMap<String, CarbonIterator<CarbonIterator<String[]>>>();
-
-  public static void put(String key, CarbonIterator<CarbonIterator<String[]>> value) {
-    iteratorMap.put(key, value);
-  }
-
-  public static CarbonIterator<CarbonIterator<String[]>> getAndRemove(String key) {
-    CarbonIterator<CarbonIterator<String[]>> iter = iteratorMap.get(key);
-    remove(key);
-    return iter;
-  }
-
-  public static void remove(String key) {
-    iteratorMap.remove(key);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInpututilsForUpdate.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInpututilsForUpdate.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInpututilsForUpdate.java
deleted file mode 100644
index b0f26e5..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/RddInpututilsForUpdate.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.csvreaderstep;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-
-public class RddInpututilsForUpdate {
-  private static Map<String, Iterator<String[]>> iteratorMap = new HashMap<String,
-      Iterator<String[]>>();
-
-  public static void put(String key, Iterator<String[]> value) {
-    iteratorMap.put(key, value);
-  }
-
-  public static Iterator<String[]> getAndRemove(String key) {
-    Iterator<String[]> iter = iteratorMap.get(key);
-    remove(key);
-    return iter;
-  }
-
-  public static void remove(String key) {
-    iteratorMap.remove(key);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6b60907/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java
deleted file mode 100644
index f99fab4..0000000
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java
+++ /dev/null
@@ -1,224 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.carbondata.processing.csvreaderstep;
-
-import java.io.BufferedReader;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.Reader;
-
-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.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-
-import com.univocity.parsers.csv.CsvParser;
-import com.univocity.parsers.csv.CsvParserSettings;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.util.LineReader;
-
-/**
- * Class which will be used to read the data from csv file and parse the record
- */
-public class UnivocityCsvParser {
-
-  private LogService LOGGER = LogServiceFactory.getLogService(this.getClass().getName());
-
-  /**
-   * Max number of columns that will be parsed for a row by univocity parsing
-   */
-  private static final int DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 2000;
-  /**
-   * Maximum allowed value for number of columns to be parsed in each row
-   */
-  private static final int THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING = 20000;
-  /**
-   * reader for csv
-   */
-  private Reader inputStreamReader;
-
-  /**
-   * buffer size of stream
-   */
-  private int bufferSize;
-
-  /**
-   * to keep track how many block has been processed
-   */
-  private int blockCounter = -1;
-
-  /**
-   * csv record parser which read and convert the record to csv format
-   */
-  private CsvParser parser;
-
-  /**
-   * row from csv
-   */
-  private String[] row;
-
-  /**
-   * holding all the properties required for parsing the records
-   */
-  private UnivocityCsvParserVo csvParserVo;
-
-  public UnivocityCsvParser(UnivocityCsvParserVo csvParserVo) {
-    this.csvParserVo = csvParserVo;
-    bufferSize = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
-            CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
-  }
-
-  /**
-   * Below method will be used to initialize the the parser
-   *
-   * @throws IOException
-   */
-  public void initialize() throws IOException {
-    CsvParserSettings parserSettings = new CsvParserSettings();
-    parserSettings.getFormat().setDelimiter(csvParserVo.getDelimiter().charAt(0));
-    parserSettings.getFormat().setComment(csvParserVo.getCommentCharacter().charAt(0));
-    parserSettings.setLineSeparatorDetectionEnabled(true);
-    parserSettings.setMaxColumns(
-        getMaxColumnsForParsing(csvParserVo.getNumberOfColumns(), csvParserVo.getMaxColumns()));
-    parserSettings.setNullValue("");
-    parserSettings.setEmptyValue("");
-    parserSettings.setIgnoreLeadingWhitespaces(false);
-    parserSettings.setIgnoreTrailingWhitespaces(false);
-    parserSettings.setSkipEmptyLines(false);
-    parserSettings.getFormat().setQuote(
-        null == csvParserVo.getQuoteCharacter() ? '\"' : csvParserVo.getQuoteCharacter().charAt(0));
-    parserSettings.getFormat().setQuoteEscape(null == csvParserVo.getEscapeCharacter() ?
-        '\\' :
-        csvParserVo.getEscapeCharacter().charAt(0));
-    blockCounter++;
-    initializeReader();
-    if (csvParserVo.getBlockDetailsList().get(blockCounter).getBlockOffset() == 0) {
-      parserSettings.setHeaderExtractionEnabled(csvParserVo.isHeaderPresent());
-    }
-    parser = new CsvParser(parserSettings);
-    parser.beginParsing(inputStreamReader);
-  }
-
-  /**
-   * This method will decide the number of columns to be parsed for a row by univocity parser
-   *
-   * @param columnCountInSchema total number of columns in schema
-   * @return
-   */
-  private int getMaxColumnsForParsing(int columnCountInSchema, int maxColumns) {
-    int maxNumberOfColumnsForParsing = DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING;
-    if (maxColumns > 0) {
-      if (columnCountInSchema >= maxColumns) {
-        // univocity parser needs one extra count from the number of columns
-        // specified during processing. eg. columnCount=12, then array size should be 13
-        maxNumberOfColumnsForParsing = columnCountInSchema + 1;
-      } else if (maxColumns > THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) {
-        maxNumberOfColumnsForParsing = THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING;
-        LOGGER.info("MAXCOLUMNS option value configured is more than system allowed limit. "
-            + "Therefore threshold value for max column parsing will be considered: "
-            + THRESHOLD_MAX_NUMBER_OF_COLUMNS_FOR_PARSING);
-      } else {
-        maxNumberOfColumnsForParsing = maxColumns;
-      }
-    } else if (columnCountInSchema >= DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING) {
-      // univocity parser needs one extra count from the number of columns
-      // specified during processing. eg. columnCount=2200, then array size should be 2201
-      maxNumberOfColumnsForParsing = columnCountInSchema + 1;
-    }
-    return maxNumberOfColumnsForParsing;
-  }
-
-  /**
-   * Below method will be used to initialize the reader
-   *
-   * @throws IOException
-   */
-  private void initializeReader() throws IOException {
-    // if already one input stream is open first we need to close and then
-    // open new stream
-    close();
-
-    String path = this.csvParserVo.getBlockDetailsList().get(blockCounter).getFilePath();
-    FileType fileType = FileFactory.getFileType(path);
-
-    if (path.endsWith(".gz") ||
-        path.endsWith(".bz2")) {
-      DataInputStream dataInputStream = FileFactory.getDataInputStream(path, fileType, bufferSize);
-      inputStreamReader = new BufferedReader(new InputStreamReader(dataInputStream));
-    } else {
-      long startOffset = this.csvParserVo.getBlockDetailsList().get(blockCounter).getBlockOffset();
-      long blockLength = this.csvParserVo.getBlockDetailsList().get(blockCounter).getBlockLength();
-      long endOffset = blockLength + startOffset;
-
-      DataInputStream dataInputStream =
-          FileFactory.getDataInputStream(path, fileType, bufferSize, startOffset);
-      // if start offset is not 0 then reading then reading and ignoring the extra line
-      if (startOffset != 0) {
-        LineReader lineReader = new LineReader(dataInputStream, 1);
-        startOffset += lineReader.readLine(new Text(), 0);
-      }
-      inputStreamReader = new BufferedReader(new InputStreamReader(
-          new BoundedDataStream(dataInputStream, endOffset - startOffset)));
-    }
-  }
-
-  /**
-   * Below method will be used to clear all the stream
-   */
-  public void close() {
-    if (null != inputStreamReader) {
-      CarbonUtil.closeStreams(inputStreamReader);
-    }
-
-  }
-
-  /**
-   * Below method will be used to check whether any more records is present or
-   * not
-   *
-   * @return true if more records are present
-   * @throws IOException
-   */
-  public boolean hasMoreRecords() throws IOException {
-    row = parser.parseNext();
-    if (row == null && blockCounter + 1 >= this.csvParserVo.getBlockDetailsList().size()) {
-      close();
-      return false;
-    }
-    if (row == null) {
-      initialize();
-      row = parser.parseNext();
-    }
-    return true;
-  }
-
-  /**
-   * Below method will be used to get the new record
-   *
-   * @return next record
-   */
-  public String[] getNextRecord() {
-    String[] returnValue = row;
-    row = null;
-    return returnValue;
-  }
-}