You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2016/09/22 05:35:59 UTC

[01/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-220] UT correction. This closes #134

Repository: incubator-carbondata
Updated Branches:
  refs/heads/branch-0.1 94447db14 -> 920b2deea


[CARBONDATA-220] UT correction. This closes #134


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

Branch: refs/heads/branch-0.1
Commit: 2b14960cc752875e803db1f3f2a6198be33e2378
Parents: f034aa3
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Thu Sep 8 13:42:38 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:19:40 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[12/50] [abbrv] incubator-carbondata git commit: fixed query issue, use different actualoffset

Posted by ra...@apache.org.
fixed query issue, use different actualoffset


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

Branch: refs/heads/branch-0.1
Commit: 32ce7910a7b92586a9c037db63a61fd50819655f
Parents: 008bee1
Author: foryou2030 <fo...@126.com>
Authored: Wed Sep 7 22:46:28 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:38:44 2016 +0530

----------------------------------------------------------------------
 .../impl/btree/BTreeDataRefNodeFinder.java         | 17 +++++++++++------
 .../spark/src/test/resources/dataWithEmptyRows.csv |  2 ++
 .../detailquery/NoDictionaryColumnTestCase.scala   | 16 ++++++++++++++++
 3 files changed, 29 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32ce7910/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java
index e443182..31273ff 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BTreeDataRefNodeFinder.java
@@ -223,6 +223,7 @@ public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
     ByteBuffer firstNoDictionaryKeyBuffer = ByteBuffer.wrap(first.getNoDictionaryKeys());
     ByteBuffer secondNoDictionaryKeyBuffer = ByteBuffer.wrap(second.getNoDictionaryKeys());
     int actualOffset = 0;
+    int actualOffset1 = 0;
     int firstNoDcitionaryLength = 0;
     int secondNodeDictionaryLength = 0;
 
@@ -237,21 +238,25 @@ public class BTreeDataRefNodeFinder implements DataRefNodeFinder {
         if (processedNoDictionaryColumn > 1) {
           actualOffset = firstNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset);
           firstNoDcitionaryLength =
-              firstNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset + SHORT_SIZE_IN_BYTES);
+              firstNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset + SHORT_SIZE_IN_BYTES)
+                      - actualOffset;
+          actualOffset1 = secondNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset);
           secondNodeDictionaryLength =
-              secondNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset + SHORT_SIZE_IN_BYTES);
+              secondNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset + SHORT_SIZE_IN_BYTES)
+                      - actualOffset1;
           compareResult = ByteUtil.UnsafeComparer.INSTANCE
-              .compareTo(first.getNoDictionaryKeys(), actualOffset, firstNoDcitionaryLength,
-                  second.getNoDictionaryKeys(), actualOffset, secondNodeDictionaryLength);
+                  .compareTo(first.getNoDictionaryKeys(), actualOffset, firstNoDcitionaryLength,
+                          second.getNoDictionaryKeys(), actualOffset1, secondNodeDictionaryLength);
           nonDictionaryKeyOffset += SHORT_SIZE_IN_BYTES;
           processedNoDictionaryColumn--;
         } else {
           actualOffset = firstNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset);
+          actualOffset1 = secondNoDictionaryKeyBuffer.getShort(nonDictionaryKeyOffset);
           firstNoDcitionaryLength = first.getNoDictionaryKeys().length - actualOffset;
-          secondNodeDictionaryLength = second.getNoDictionaryKeys().length - actualOffset;
+          secondNodeDictionaryLength = second.getNoDictionaryKeys().length - actualOffset1;
           compareResult = ByteUtil.UnsafeComparer.INSTANCE
               .compareTo(first.getNoDictionaryKeys(), actualOffset, firstNoDcitionaryLength,
-                  second.getNoDictionaryKeys(), actualOffset, secondNodeDictionaryLength);
+                  second.getNoDictionaryKeys(), actualOffset1, secondNodeDictionaryLength);
         }
       }
       if (compareResult != 0) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32ce7910/integration/spark/src/test/resources/dataWithEmptyRows.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/dataWithEmptyRows.csv b/integration/spark/src/test/resources/dataWithEmptyRows.csv
new file mode 100644
index 0000000..0e3a81f
--- /dev/null
+++ b/integration/spark/src/test/resources/dataWithEmptyRows.csv
@@ -0,0 +1,2 @@
+29000,cust_name_2000,active_emui_version_2000,2010-10-04 01:00:01,12345678
+,,,,0

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/32ce7910/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
index 1b3a4bc..b36da1f 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
@@ -33,10 +33,17 @@ class NoDictionaryColumnTestCase extends QueryTest with BeforeAndAfterAll {
   override def beforeAll {
     sql("DROP TABLE IF EXISTS carbonTable")
     sql("DROP TABLE IF EXISTS hiveTable")
+    sql("DROP TABLE IF EXISTS carbonEmpty")
+    sql("DROP TABLE IF EXISTS hiveEmpty")
     sql("CREATE TABLE carbonTable (imei String, age Int, num BigInt) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='age,num')")
     sql("LOAD DATA LOCAL INPATH './src/test/resources/datawithNegtiveNumber.csv' INTO TABLE carbonTable")
     sql("CREATE TABLE hiveTable (imei String, age Int, num BigInt) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','")
     sql("LOAD DATA LOCAL INPATH './src/test/resources/datawithNegeativewithoutHeader.csv' INTO TABLE hiveTable")
+
+    sql("CREATE TABLE carbonEmpty (cust_id int, cust_name String, active_emui_version String, bob timestamp, bigint_column bigint) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='cust_name,active_emui_version')")
+    sql("LOAD DATA LOCAL INPATH './src/test/resources/dataWithEmptyRows.csv' INTO TABLE carbonEmpty OPTIONS('FILEHEADER'='cust_id,cust_name,active_emui_version,bob,bigint_column')")
+    sql("CREATE TABLE hiveEmpty (cust_id int, cust_name String, active_emui_version String, bob timestamp, bigint_column bigint) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','")
+    sql("LOAD DATA LOCAL INPATH './src/test/resources/dataWithEmptyRows.csv' INTO TABLE hiveEmpty")
   }
 
   test("SELECT IntType FROM carbonTable") {
@@ -53,8 +60,17 @@ class NoDictionaryColumnTestCase extends QueryTest with BeforeAndAfterAll {
     )
   }
 
+  test("test load data with one row that all no dictionary column values are empty") {
+    checkAnswer(
+      sql("SELECT cust_name,active_emui_version FROM carbonEmpty"),
+      sql("SELECT cust_name,active_emui_version FROM hiveEmpty")
+    )
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS carbonTable")
     sql("DROP TABLE IF EXISTS hiveTable")
+    sql("DROP TABLE IF EXISTS carbonEmpty")
+    sql("DROP TABLE IF EXISTS hiveEmpty")
   }
 }
\ No newline at end of file


[04/50] [abbrv] incubator-carbondata git commit: adapt data with header for all dictionary

Posted by ra...@apache.org.
adapt data with header for all dictionary

use DEFAULT_CHARSET

remove listFiles


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

Branch: refs/heads/branch-0.1
Commit: 51e4c11e40611494a456a806a86d57d9348b5b4f
Parents: 854b75e
Author: foryou2030 <fo...@126.com>
Authored: Mon Aug 22 18:00:00 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:29:22 2016 +0530

----------------------------------------------------------------------
 .../apache/carbondata/core/util/CarbonUtil.java | 30 ++++++++++++
 .../src/main/resources/datawithoutheader.csv    | 10 ----
 .../examples/AllDictionaryExample.scala         | 12 ++---
 .../examples/util/AllDictionaryUtil.scala       |  2 +-
 .../spark/util/GlobalDictionaryUtil.scala       | 50 +++++++++++++-------
 .../processing/csvload/GraphExecutionUtil.java  | 29 +-----------
 6 files changed, 71 insertions(+), 62 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/51e4c11e/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 2460f6e..df538e0 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
@@ -20,14 +20,17 @@
 
 package org.apache.carbondata.core.util;
 
+import java.io.BufferedReader;
 import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStreamReader;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.nio.charset.Charset;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -1385,5 +1388,32 @@ public final class CarbonUtil {
     }
     return dictionaryOneChunkSize;
   }
+
+  /**
+   * @param csvFilePath
+   * @return
+   */
+  public static String readHeader(String csvFilePath) {
+
+    DataInputStream fileReader = null;
+    BufferedReader bufferedReader = null;
+    String readLine = null;
+
+    try {
+      fileReader =
+          FileFactory.getDataInputStream(csvFilePath, FileFactory.getFileType(csvFilePath));
+      bufferedReader = new BufferedReader(new InputStreamReader(fileReader,
+              Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+      readLine = bufferedReader.readLine();
+
+    } catch (FileNotFoundException e) {
+      LOGGER.error(e, "CSV Input File not found  " + e.getMessage());
+    } catch (IOException e) {
+      LOGGER.error(e, "Not able to read CSV input File  " + e.getMessage());
+    } finally {
+      CarbonUtil.closeStreams(fileReader, bufferedReader);
+    }
+    return readLine;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/51e4c11e/examples/src/main/resources/datawithoutheader.csv
----------------------------------------------------------------------
diff --git a/examples/src/main/resources/datawithoutheader.csv b/examples/src/main/resources/datawithoutheader.csv
deleted file mode 100644
index df2b945..0000000
--- a/examples/src/main/resources/datawithoutheader.csv
+++ /dev/null
@@ -1,10 +0,0 @@
-1,2015/7/23,china,aaa1,phone197,ASD69643,15000
-2,2015/7/24,china,aaa2,phone756,ASD42892,15001
-3,2015/7/25,china,aaa3,phone1904,ASD37014,15002
-4,2015/7/26,china,aaa4,phone2435,ASD66902,15003
-5,2015/7/27,china,aaa5,phone2441,ASD90633,15004
-6,2015/7/28,china,aaa6,phone294,ASD59961,15005
-7,2015/7/29,china,aaa7,phone610,ASD14875,15006
-8,2015/7/30,china,aaa8,phone1848,ASD57308,15007
-9,2015/7/18,china,aaa9,phone706,ASD86717,15008
-10,2015/7/19,usa,aaa10,phone685,ASD30505,15009

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/51e4c11e/examples/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala b/examples/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
index 195c7a6..a2b72e3 100644
--- a/examples/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
+++ b/examples/src/main/scala/org/apache/carbondata/examples/AllDictionaryExample.scala
@@ -23,11 +23,10 @@ import org.apache.carbondata.examples.util.{AllDictionaryUtil, InitForExamples}
 object AllDictionaryExample {
   def main(args: Array[String]) {
     val cc = InitForExamples.createCarbonContext("CarbonExample")
-    val testData = InitForExamples.currentPath + "/src/main/resources/datawithoutheader.csv"
-    val csvHeader = "id,date,country,name,phonetype,serialname,salary"
+    val testData = InitForExamples.currentPath + "/src/main/resources/data.csv"
+    val csvHeader = "ID,date,country,name,phonetype,serialname,salary"
     val dictCol = "|date|country|name|phonetype|serialname|"
-    val allDictFile = InitForExamples.currentPath +
-      "/src/main/resources/datawithoutheader.dictionary"
+    val allDictFile = InitForExamples.currentPath + "/src/main/resources/data.dictionary"
     // extract all dictionary files from source data
     AllDictionaryUtil.extractDictionary(cc.sparkContext,
       testData, allDictFile, csvHeader, dictCol)
@@ -41,13 +40,12 @@ object AllDictionaryExample {
            CREATE TABLE IF NOT EXISTS t3
            (ID Int, date Timestamp, country String,
            name String, phonetype String, serialname String, salary Int)
-           STORED BY 'org.apache.carbondata.format'
+           STORED BY 'carbondata'
            """)
 
     cc.sql(s"""
            LOAD DATA LOCAL INPATH '$testData' into table t3
-           options('FILEHEADER'='id,date,country,name,phonetype,serialname,salary',
-           'ALL_DICTIONARY_PATH'='$allDictFile')
+           options('ALL_DICTIONARY_PATH'='$allDictFile')
            """)
 
     cc.sql("""

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/51e4c11e/examples/src/main/scala/org/apache/carbondata/examples/util/AllDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/carbondata/examples/util/AllDictionaryUtil.scala b/examples/src/main/scala/org/apache/carbondata/examples/util/AllDictionaryUtil.scala
index 3e8df71..bd625f3 100644
--- a/examples/src/main/scala/org/apache/carbondata/examples/util/AllDictionaryUtil.scala
+++ b/examples/src/main/scala/org/apache/carbondata/examples/util/AllDictionaryUtil.scala
@@ -35,7 +35,7 @@ object AllDictionaryUtil extends Logging{
     val fileHeaderArr = fileHeader.split(",")
     val isDictCol = new Array[Boolean](fileHeaderArr.length)
     for (i <- 0 until fileHeaderArr.length) {
-      if (dictCol.contains("|" + fileHeaderArr(i) + "|")) {
+      if (dictCol.contains("|" + fileHeaderArr(i).toLowerCase() + "|")) {
         isDictCol(i) = true
       } else {
         isDictCol(i) = false

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/51e4c11e/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index ddb596b..b96a826 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -43,7 +43,7 @@ import org.apache.carbondata.core.carbon.path.CarbonStorePath
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory
 import org.apache.carbondata.core.reader.CarbonDictionaryReader
-import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.apache.carbondata.core.writer.CarbonDictionaryWriter
 import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.carbondata.spark.CarbonSparkFactory
@@ -583,8 +583,8 @@ object GlobalDictionaryUtil extends Logging {
    * @return allDictionaryRdd
    */
   private def readAllDictionaryFiles(sqlContext: SQLContext,
-                                       csvFileColumns: Array[String],
-                                       requireColumns: Array[String],
+                                     csvFileColumns: Array[String],
+                                     requireColumns: Array[String],
                                      allDictionaryPath: String) = {
     var allDictionaryRdd: RDD[(String, Iterable[String])] = null
     try {
@@ -651,6 +651,31 @@ object GlobalDictionaryUtil extends Logging {
   }
 
   /**
+   * get file headers from fact file
+   *
+   * @param carbonLoadModel
+   * @return headers
+   */
+  private def getHeaderFormFactFile(carbonLoadModel: CarbonLoadModel): Array[String] = {
+    var headers: Array[String] = null
+    val factFile: String = carbonLoadModel.getFactFilePath.split(",")(0)
+    val readLine = CarbonUtil.readHeader(factFile)
+
+    if (null != readLine) {
+      val delimiter = if (StringUtils.isEmpty(carbonLoadModel.getCsvDelimiter)) {
+        "" + CSVWriter.DEFAULT_SEPARATOR
+      } else {
+        carbonLoadModel.getCsvDelimiter
+      }
+      headers = readLine.toLowerCase().split(delimiter);
+    } else {
+      logError("Not found file header! Please set fileheader")
+      throw new IOException("Failed to get file header")
+    }
+    headers
+  }
+
+  /**
    * generate global dictionary with SQLContext and CarbonLoadModel
    *
    * @param sqlContext  sql context
@@ -736,27 +761,20 @@ object GlobalDictionaryUtil extends Logging {
         logInfo("Generate global dictionary from all dictionary files!")
         val isNonempty = validateAllDictionaryPath(allDictionaryPath)
         if(isNonempty) {
-          // fill the map[columnIndex -> columnName]
-          var fileHeaders : Array[String] = null
-          if(!StringUtils.isEmpty(carbonLoadModel.getCsvHeader)) {
-            val splitColumns = carbonLoadModel.getCsvHeader.split("" + CSVWriter.DEFAULT_SEPARATOR)
-            val fileHeadersArr = new ArrayBuffer[String]()
-            for(i <- 0 until splitColumns.length) {
-              fileHeadersArr += splitColumns(i).trim.toLowerCase()
-            }
-            fileHeaders = fileHeadersArr.toArray
+          var headers = if (StringUtils.isEmpty(carbonLoadModel.getCsvHeader)) {
+            getHeaderFormFactFile(carbonLoadModel)
           } else {
-            logError("Not found file header! Please set fileheader")
-            throw new IOException("Failed to get file header")
+            carbonLoadModel.getCsvHeader.toLowerCase.split("" + CSVWriter.DEFAULT_SEPARATOR)
           }
+          headers = headers.map(headerName => headerName.trim)
           // prune columns according to the CSV file header, dimension columns
           val (requireDimension, requireColumnNames) =
-            pruneDimensions(dimensions, fileHeaders, fileHeaders)
+            pruneDimensions(dimensions, headers, headers)
           if (requireDimension.nonEmpty) {
             val model = createDictionaryLoadModel(carbonLoadModel, table, requireDimension,
               hdfsLocation, dictfolderPath, false)
             // read local dictionary file, and group by key
-            val allDictionaryRdd = readAllDictionaryFiles(sqlContext, fileHeaders,
+            val allDictionaryRdd = readAllDictionaryFiles(sqlContext, headers,
               requireColumnNames, allDictionaryPath)
             // read exist dictionary and combine
             val inputRDD = new CarbonAllDictionaryCombineRDD(allDictionaryRdd, model)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/51e4c11e/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
index 2a35002..6d82bcd 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/GraphExecutionUtil.java
@@ -254,40 +254,13 @@ public final class GraphExecutionUtil {
 
   /**
    * @param csvFilePath
-   * @return
-   */
-  private static String readCSVFile(String csvFilePath) {
-
-    DataInputStream fileReader = null;
-    BufferedReader bufferedReader = null;
-    String readLine = null;
-
-    try {
-      fileReader =
-          FileFactory.getDataInputStream(csvFilePath, FileFactory.getFileType(csvFilePath));
-      bufferedReader =
-          new BufferedReader(new InputStreamReader(fileReader, Charset.defaultCharset()));
-      readLine = bufferedReader.readLine();
-
-    } catch (FileNotFoundException e) {
-      LOGGER.error(e, "CSV Input File not found  " + e.getMessage());
-    } catch (IOException e) {
-      LOGGER.error(e, "Not able to read CSV input File  " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(fileReader, bufferedReader);
-    }
-    return readLine;
-  }
-
-  /**
-   * @param csvFilePath
    * @param columnNames
    * @return
    */
   public static boolean checkCSVAndRequestedTableColumns(String csvFilePath, String[] columnNames,
       String delimiter) {
 
-    String readLine = readCSVFile(csvFilePath);
+    String readLine = CarbonUtil.readHeader(csvFilePath);
 
     if (null != readLine) {
       delimiter = CarbonUtil.delimiterConverter(delimiter);


[43/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-254] Code Inspection This closes #171

Posted by ra...@apache.org.
[CARBONDATA-254] Code Inspection This closes #171


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

Branch: refs/heads/branch-0.1
Commit: 17b00c27397d9a1dff8c6cb20b440b90af74d21d
Parents: 5928bb9
Author: jackylk <ja...@huawei.com>
Authored: Sun Sep 18 21:44:46 2016 -0700
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:34:46 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[09/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-227] defaultparallism fix for dynamic executors. This closes #140

Posted by ra...@apache.org.
[CARBONDATA-227] defaultparallism fix for dynamic executors. This closes #140


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

Branch: refs/heads/branch-0.1
Commit: 7e3d46ce08881faa2e7630d4fcc1916a05c0d499
Parents: acb1d97
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Thu Sep 8 14:25:06 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:34:27 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[35/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-173]The exception info is not proper when measure used as columnDict This closes #88

Posted by ra...@apache.org.
[CARBONDATA-173]The exception info is not proper when measure used as columnDict This closes #88


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

Branch: refs/heads/branch-0.1
Commit: b53a4db4f42033110ee5b23985c765295c9b8daa
Parents: 1653327
Author: chenliang613 <ch...@apache.org>
Authored: Mon Aug 29 22:49:04 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:29:09 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[10/50] [abbrv] incubator-carbondata git commit: Problem: Array Index of bound exception thrown from dictionary look up while writing sort index file

Posted by ra...@apache.org.
Problem: Array Index of bound exception thrown from dictionary look up while writing sort index file

Analysis: Whenever we load dictionary data into memory, then in case of populating reverse dictionary object sometimes a chunk which has no value is also getting added to the dictionary chunk list. This is happening because the logic for dictionary chunk distribution in case of forward dictionary is not implemented for reverse dictionary and 0 size dictionary chunks are not getting removed while adding to the list of dictionary chunks.

Solution: Add the same distribution logic we have in forward dictionary for populating reverse dictionary object

Impact area: Sort index generation


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

Branch: refs/heads/branch-0.1
Commit: ad14039ef4e298345e48c9c4985754d6d5ae2c50
Parents: 7e3d46c
Author: manishgupta88 <to...@gmail.com>
Authored: Thu Sep 8 13:08:56 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:35:09 2016 +0530

----------------------------------------------------------------------
 .../core/cache/dictionary/DictionaryCacheLoaderImpl.java       | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ad14039e/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
index e798c01..ce3107b 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
@@ -95,7 +95,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
     int dictionaryChunkSize = CarbonUtil.getDictionaryChunkSize();
     int sizeOfLastDictionaryChunk = dictionaryInfo.getSizeOfLastDictionaryChunk();
     int sizeOfOneDictionaryChunk = dictionaryChunkSize - sizeOfLastDictionaryChunk;
-    if (sizeOfOneDictionaryChunk == 0) {
+    if (sizeOfOneDictionaryChunk <= 0) {
       sizeOfOneDictionaryChunk = dictionaryChunkSize;
     }
     List<List<byte[]>> dictionaryChunks =
@@ -111,7 +111,9 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
       }
     }
     for (List<byte[]> dictionaryChunk : dictionaryChunks) {
-      dictionaryInfo.addDictionaryChunk(dictionaryChunk);
+      if (!dictionaryChunk.isEmpty()) {
+        dictionaryInfo.addDictionaryChunk(dictionaryChunk);
+      }
     }
   }
 


[14/50] [abbrv] incubator-carbondata git commit: Fixed data mismatch issue in case of Dictionary Exclude column for Numeric data type

Posted by ra...@apache.org.
Fixed data mismatch issue in case of Dictionary Exclude column for Numeric data type


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

Branch: refs/heads/branch-0.1
Commit: 7cfc3ec28df89d6c4de20e5d2804bf075c54e4d8
Parents: 6411fde
Author: kumarvishal <ku...@gmail.com>
Authored: Wed Sep 7 22:16:17 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:39:49 2016 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  6 +--
 .../AllDataTypesTestCaseAggregate.scala         |  2 +-
 .../createtable/TestCreateTableSyntax.scala     | 14 ++++++
 .../dataload/TestLoadDataWithHiveSyntax.scala   | 36 ++++++-------
 .../deleteTable/TestDeleteTableNewDDL.scala     |  6 +--
 .../HighCardinalityDataTypesTestCase.scala      | 32 ++++++------
 .../NoDictionaryColumnTestCase.scala            |  2 +-
 .../apache/spark/sql/TestCarbonSqlParser.scala  | 53 ++++++++++----------
 8 files changed, 82 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cfc3ec2/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index 3bc5f5c..69d921f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -773,7 +773,7 @@ class CarbonSqlParser()
               val errormsg = "DICTIONARY_EXCLUDE is unsupported for complex datatype column: " +
                 dictExcludeCol
               throw new MalformedCarbonCommandException(errormsg)
-            } else if (isDoubleDecimalColDictionaryExclude(dataType)) {
+            } else if (!isStringAndTimestampColDictionaryExclude(dataType)) {
               val errorMsg = "DICTIONARY_EXCLUDE is unsupported for " + dataType.toLowerCase() +
                 " data type column: " + dictExcludeCol
               throw new MalformedCarbonCommandException(errorMsg)
@@ -862,8 +862,8 @@ class CarbonSqlParser()
    /**
     * detects whether double or decimal column is part of dictionary_exclude
     */
-  def isDoubleDecimalColDictionaryExclude(columnDataType: String): Boolean = {
-    val dataTypes = Array("double", "decimal")
+  def isStringAndTimestampColDictionaryExclude(columnDataType: String): Boolean = {
+    val dataTypes = Array("string", "timestamp")
     dataTypes.exists(x => x.equalsIgnoreCase(columnDataType))
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cfc3ec2/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
index 7d7a4e4..d4def47 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
@@ -40,7 +40,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll {
     val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../")
       .getCanonicalPath
 
-    sql("create table if not exists Carbon_automation_test (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active
 _phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string, gamePointId int,contractNumber int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Latest_MONTH,Latest_DAY,deviceInformationId')");
+    sql("create table if not exists Carbon_automation_test (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active
 _phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string, gamePointId int,contractNumber int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='Latest_MONTH,Latest_DAY,deviceInformationId')");
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
     sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/100_olap.csv' INTO table Carbon_automation_test options('DELIMITER'= ',' ,'QUOTECHAR'= '\"', 'FILEHEADER'= 'imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Late
 st_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')");

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cfc3ec2/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createtable/TestCreateTableSyntax.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createtable/TestCreateTableSyntax.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createtable/TestCreateTableSyntax.scala
index 6fd5e1d..b99e73f 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createtable/TestCreateTableSyntax.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/createtable/TestCreateTableSyntax.scala
@@ -94,6 +94,20 @@ class TestCreateTableSyntax extends QueryTest with BeforeAndAfterAll {
     }
     sql("drop table if exists carbontable")
   }
+    test("test carbon table create with int datatype as dictionary exclude") {
+    try {
+      sql("create table carbontable(id int, name string, dept string, mobile array<string>, "+
+        "country string, salary double) STORED BY 'org.apache.carbondata.format' " +
+        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='id')")
+      assert(false)
+    } catch {
+      case e : MalformedCarbonCommandException => {
+        assert(e.getMessage.equals("DICTIONARY_EXCLUDE is unsupported for int " +
+          "data type column: id"))
+      }
+    }
+    sql("drop table if exists carbontable")
+  }
 
   test("test carbon table create with decimal datatype as dictionary exclude") {
     try {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cfc3ec2/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
index da64b39..e017a62 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
@@ -55,17 +55,17 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
   test("test data loading and validate query output") {
     //Create test cube and hive table
     sql(
-      "CREATE table testtable (empno int, empname String, designation String, doj String, " +
-        "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
-        "projectcode int, projectjoindate String, projectenddate String,attendance double," +
+      "CREATE table testtable (empno string, empname String, designation String, doj String, " +
+        "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
+        "projectcode string, projectjoindate String, projectenddate String,attendance double," +
         "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
         "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
         "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
     )
     sql(
-      "create table testhivetable(empno int, empname String, designation string, doj String, " +
-        "workgroupcategory int, workgroupcategoryname String,deptno int, deptname String, " +
-        "projectcode int, projectjoindate String,projectenddate String, attendance double," +
+      "create table testhivetable(empno string, empname String, designation string, doj String, " +
+        "workgroupcategory string, workgroupcategoryname String,deptno string, deptname String, " +
+        "projectcode string, projectjoindate String,projectenddate String, attendance double," +
         "utilization double,salary double)row format delimited fields terminated by ','"
     )
     //load data into test cube and hive table and validate query result
@@ -96,17 +96,17 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
   test("test data loading with different case file header and validate query output") {
     //Create test cube and hive table
     sql(
-      "CREATE table testtable1 (empno int, empname String, designation String, doj String, " +
-        "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
-        "projectcode int, projectjoindate String, projectenddate String,attendance double," +
+      "CREATE table testtable1 (empno string, empname String, designation String, doj String, " +
+        "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
+        "projectcode string, projectjoindate String, projectenddate String,attendance double," +
         "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
         "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
         "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
     )
     sql(
-      "create table testhivetable1(empno int, empname String, designation string, doj String, " +
-        "workgroupcategory int, workgroupcategoryname String,deptno int, deptname String, " +
-        "projectcode int, projectjoindate String,projectenddate String, attendance double," +
+      "create table testhivetable1(empno string, empname String, designation string, doj String, " +
+        "workgroupcategory string, workgroupcategoryname String,deptno string, deptname String, " +
+        "projectcode string, projectjoindate String,projectenddate String, attendance double," +
         "utilization double,salary double)row format delimited fields terminated by ','"
     )
     //load data into test cube and hive table and validate query result
@@ -566,17 +566,17 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
 
   test("test data loading when delimiter is '|' and data with header") {
     sql(
-      "CREATE table carbontable1 (empno int, empname String, designation String, doj String, " +
-        "workgroupcategory int, workgroupcategoryname String, deptno int, deptname String, " +
-        "projectcode int, projectjoindate String, projectenddate String,attendance double," +
+      "CREATE table carbontable1 (empno string, empname String, designation String, doj String, " +
+        "workgroupcategory string, workgroupcategoryname String, deptno string, deptname String, " +
+        "projectcode string, projectjoindate String, projectenddate String,attendance double," +
         "utilization double,salary double) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES" +
         "('DICTIONARY_EXCLUDE'='empno,empname,designation,doj,workgroupcategory," +
         "workgroupcategoryname,deptno,deptname,projectcode,projectjoindate,projectenddate')"
     )
     sql(
-      "create table hivetable1 (empno int, empname String, designation string, doj String, " +
-        "workgroupcategory int, workgroupcategoryname String,deptno int, deptname String, " +
-        "projectcode int, projectjoindate String,projectenddate String, attendance double," +
+      "create table hivetable1 (empno string, empname String, designation string, doj String, " +
+        "workgroupcategory string, workgroupcategoryname String,deptno string, deptname String, " +
+        "projectcode string, projectjoindate String,projectenddate String, attendance double," +
         "utilization double,salary double)row format delimited fields terminated by ','"
     )
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cfc3ec2/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
index bd822e4..67c2d4a 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
@@ -159,7 +159,7 @@ class TestDeleteTableNewDDL extends QueryTest with BeforeAndAfterAll {
       "CREATE table dropTableTest2 (ID int, date String, country String, name " +
       "String," +
       "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES('DICTIONARY_EXCLUDE'='salary')"
+      "TBLPROPERTIES('DICTIONARY_INCLUDE'='salary')"
     )
     sql(
       "LOAD DATA LOCAL INPATH '" + resource + "dataretention1.csv' INTO TABLE dropTableTest2 " +
@@ -170,7 +170,7 @@ class TestDeleteTableNewDDL extends QueryTest with BeforeAndAfterAll {
       "CREATE table dropTableTest2 (ID int, date String, country String, name " +
       "String," +
       "phonetype String, serialname String, salary decimal) stored by 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES('DICTIONARY_EXCLUDE'='date')"
+      "TBLPROPERTIES('DICTIONARY_INCLUDE'='date')"
     )
     sql(
       "LOAD DATA LOCAL INPATH '" + resource + "dataretention1.csv' INTO TABLE dropTableTest2 " +
@@ -185,7 +185,7 @@ class TestDeleteTableNewDDL extends QueryTest with BeforeAndAfterAll {
       "CREATE table test.dropTableTest3 (ID int, date String, country String, name " +
       "String," +
       "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format' " +
-      "TBLPROPERTIES('DICTIONARY_EXCLUDE'='salary')"
+      "TBLPROPERTIES('DICTIONARY_INCLUDE'='salary')"
     )
     sql(
       "LOAD DATA LOCAL INPATH '" + resource + "dataretention1.csv' INTO TABLE test.dropTableTest3 " +

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cfc3ec2/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
index f349150..062c055 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
@@ -42,7 +42,7 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists NO_DICTIONARY_CARBON_7")
     
     sql(
-      "create table NO_DICTIONARY_HIVE_6(empno int,empname string,designation string,doj " +
+      "create table NO_DICTIONARY_HIVE_6(empno string,empname string,designation string,doj " +
         "Timestamp,workgroupcategory int, " +
         "workgroupcategoryname string,deptno int, deptname string, projectcode int, " +
         "projectjoindate Timestamp,projectenddate Timestamp,attendance int, "
@@ -55,12 +55,12 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
         "NO_DICTIONARY_HIVE_6"
     );
     //For Carbon cube creation.
-    sql("CREATE TABLE NO_DICTIONARY_CARBON_6 (empno Int, " +
+    sql("CREATE TABLE NO_DICTIONARY_CARBON_6 (empno string, " +
       "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
       "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
       "projectenddate Timestamp, designation String,attendance Int,utilization " +
       "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
-        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno,empname,designation')"
+        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empname,designation')"
     )
     sql(
       "LOAD DATA LOCAL INPATH './src/test/resources/data.csv' INTO TABLE NO_DICTIONARY_CARBON_6 " +
@@ -78,7 +78,7 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
       "LOAD DATA LOCAL INPATH './src/test/resources/data.csv' INTO TABLE NO_DICTIONARY_CARBON_7 " +
       "OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
-    sql("CREATE TABLE filtertestTable (ID Int,date Timestamp, country String, " +
+    sql("CREATE TABLE filtertestTable (ID string,date Timestamp, country String, " +
       "name String, phonetype String, serialname String, salary Int) " +
         "STORED BY 'org.apache.carbondata.format' " +  "TBLPROPERTIES('DICTIONARY_EXCLUDE'='ID')"
     )
@@ -94,7 +94,7 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
 
   test("Count (*) with filter") {
     checkAnswer(
-      sql("select count(*) from NO_DICTIONARY_CARBON_6 where empno=11"),
+      sql("select count(*) from NO_DICTIONARY_CARBON_6 where empno='11'"),
       Seq(Row(1))
     )
   }
@@ -104,7 +104,7 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
 
     checkAnswer(
       sql("select empno from NO_DICTIONARY_CARBON_6"),
-      Seq(Row(11), Row(12), Row(13), Row(14), Row(15), Row(16), Row(17), Row(18), Row(19), Row(20))
+      Seq(Row("11"), Row("12"), Row("13"), Row("14"), Row("15"), Row("16"), Row("17"), Row("18"), Row("19"), Row("20"))
     )
 
 
@@ -132,16 +132,16 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
 
 
     checkAnswer(
-      sql("select empno from NO_DICTIONARY_CARBON_6 where empno in(11,12,13)"),
-      Seq(Row(11), Row(12), Row(13))
+      sql("select empno from NO_DICTIONARY_CARBON_6 where empno in('11','12','13')"),
+      Seq(Row("11"), Row("12"), Row("13"))
     )
   }
   test("Detail Query with NO_DICTIONARY_COLUMN with not in filter Compare With HIVE RESULT") {
 
 
     checkAnswer(
-      sql("select empno from NO_DICTIONARY_CARBON_6 where empno not in(11,12,13,14,15,16,17)"),
-      Seq(Row(18), Row(19), Row(20))
+      sql("select empno from NO_DICTIONARY_CARBON_6 where empno not in('11','12','13','14','15','16','17')"),
+      Seq(Row("18"), Row("19"), Row("20"))
     )
   }
 
@@ -149,8 +149,8 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
 
 
     checkAnswer(
-      sql("select empno from NO_DICTIONARY_CARBON_6 where empno=17"),
-      Seq(Row(17))
+      sql("select empno from NO_DICTIONARY_CARBON_6 where empno='17'"),
+      Seq(Row("17"))
     )
   }
   test("Detail Query with NO_DICTIONARY_COLUMN with IS NOT NULL filter") {
@@ -158,13 +158,13 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
 
     checkAnswer(
       sql("select id  from filtertestTable where id is not null"),
-      Seq(Row(4), Row(6))
+      Seq(Row("4"),Row("6"),Row("abc"))
     )
   }
 test("filter with arithmetic expression") {
     checkAnswer(
       sql("select id from filtertestTable " + "where id+2 = 6"),
-      Seq(Row(4))
+      Seq(Row("4"))
     )
   }
   test("Detail Query with NO_DICTIONARY_COLUMN with equals multiple filter Compare With HIVE " +
@@ -173,8 +173,8 @@ test("filter with arithmetic expression") {
 
 
     checkAnswer(
-      sql("select empno,empname,workgroupcategory from NO_DICTIONARY_CARBON_6 where empno=17"),
-      sql("select empno,empname,workgroupcategory from NO_DICTIONARY_HIVE_6 where empno=17")
+      sql("select empno,empname,workgroupcategory from NO_DICTIONARY_CARBON_6 where empno='17'"),
+      sql("select empno,empname,workgroupcategory from NO_DICTIONARY_HIVE_6 where empno='17'")
     )
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cfc3ec2/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
index b36da1f..9ed969a 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/detailquery/NoDictionaryColumnTestCase.scala
@@ -35,7 +35,7 @@ class NoDictionaryColumnTestCase extends QueryTest with BeforeAndAfterAll {
     sql("DROP TABLE IF EXISTS hiveTable")
     sql("DROP TABLE IF EXISTS carbonEmpty")
     sql("DROP TABLE IF EXISTS hiveEmpty")
-    sql("CREATE TABLE carbonTable (imei String, age Int, num BigInt) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='age,num')")
+    sql("CREATE TABLE carbonTable (imei String, age Int, num BigInt) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_INCLUDE'='age,num')")
     sql("LOAD DATA LOCAL INPATH './src/test/resources/datawithNegtiveNumber.csv' INTO TABLE carbonTable")
     sql("CREATE TABLE hiveTable (imei String, age Int, num BigInt) ROW FORMAT DELIMITED FIELDS TERMINATED BY ','")
     sql("LOAD DATA LOCAL INPATH './src/test/resources/datawithNegeativewithoutHeader.csv' INTO TABLE hiveTable")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7cfc3ec2/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala b/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
index e18e0ed..04e47bb 100644
--- a/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
@@ -65,10 +65,10 @@ class TestCarbonSqlParser extends QueryTest {
   def loadAllFields: Seq[Field] = {
     var fields: Seq[Field] = Seq[Field]()
 
-    var col1 = Field("col1", Option("Int"), Option("col1"), None, null, Some("columnar"))
+    var col1 = Field("col1", Option("String"), Option("col1"), None, null, Some("columnar"))
     var col2 = Field("col2", Option("String"), Option("col2"), None, null, Some("columnar"))
     var col3 = Field("col3", Option("String"), Option("col3"), None, null, Some("columnar"))
-    var col4 = Field("col4", Option("Int"), Option("col4"), None, null, Some("columnar"))
+    var col4 = Field("col4", Option("int"), Option("col4"), None, null, Some("columnar"))
     var col5 = Field("col5", Option("String"), Option("col5"), None, null, Some("columnar"))
     var col6 = Field("col6", Option("String"), Option("col6"), None, null, Some("columnar"))
     var col7 = Field("col7", Option("String"), Option("col7"), None, null, Some("columnar"))
@@ -203,10 +203,11 @@ class TestCarbonSqlParser extends QueryTest {
     // testing col
 
     //All dimension fields should be available in dimensions list
-    assert(dimCols.size == 7)
-    assert(dimCols.lift(0).get.column.equalsIgnoreCase("col2"))
-    assert(dimCols.lift(1).get.column.equalsIgnoreCase("col3"))
-    assert(dimCols.lift(2).get.column.equalsIgnoreCase("col4"))
+    assert(dimCols.size == 8)
+    assert(dimCols.lift(0).get.column.equalsIgnoreCase("col1"))
+    assert(dimCols.lift(1).get.column.equalsIgnoreCase("col2"))
+    assert(dimCols.lift(2).get.column.equalsIgnoreCase("col3"))
+    assert(dimCols.lift(3).get.column.equalsIgnoreCase("col4"))
 
     //No dictionary column names will be available in noDictionary list
     assert(noDictionary.size == 1)
@@ -290,22 +291,22 @@ class TestCarbonSqlParser extends QueryTest {
     val msrCols = stub.extractMsrColsFromFieldsTest(fields, tableProperties)
 
     //below dimension fields should be available in dimensions list
-    assert(dimCols.size == 6)
-    assert(dimCols.lift(0).get.column.equalsIgnoreCase("col2"))
-    assert(dimCols.lift(1).get.column.equalsIgnoreCase("col3"))
+    assert(dimCols.size == 7)
+    assert(dimCols.lift(0).get.column.equalsIgnoreCase("col1"))
+    assert(dimCols.lift(1).get.column.equalsIgnoreCase("col2"))
+    assert(dimCols.lift(2).get.column.equalsIgnoreCase("col3"))
 
     //below column names will be available in noDictionary list
     assert(noDictionary.size == 1)
     assert(noDictionary.lift(0).get.equalsIgnoreCase("col3"))
 
     //check msr
-    assert(msrCols.size == 2)
-    assert(msrCols.lift(0).get.column.equalsIgnoreCase("col1"))
-    assert(msrCols.lift(1).get.column.equalsIgnoreCase("col4"))
+    assert(msrCols.size == 1)
+    assert(msrCols.lift(0).get.column.equalsIgnoreCase("col4"))
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields5") {
-    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col4", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col2")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col1", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col2")
     val fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     val (dimCols, noDictionary) = stub
@@ -314,17 +315,17 @@ class TestCarbonSqlParser extends QueryTest {
 
     //below dimension fields should be available in dimensions list
     assert(dimCols.size == 7)
-    assert(dimCols.lift(0).get.column.equalsIgnoreCase("col2"))
-    assert(dimCols.lift(1).get.column.equalsIgnoreCase("col3"))
-    assert(dimCols.lift(2).get.column.equalsIgnoreCase("col4"))
+    assert(dimCols.lift(0).get.column.equalsIgnoreCase("col1"))
+    assert(dimCols.lift(1).get.column.equalsIgnoreCase("col2"))
+    assert(dimCols.lift(2).get.column.equalsIgnoreCase("col3"))
 
     //below column names will be available in noDictionary list
     assert(noDictionary.size == 1)
-    assert(noDictionary.lift(0).get.equalsIgnoreCase("col4"))
+    assert(noDictionary.lift(0).get.equalsIgnoreCase("col1"))
 
     //check msr
     assert(msrCols.size == 1)
-    assert(msrCols.lift(0).get.column.equalsIgnoreCase("col1"))
+    assert(msrCols.lift(0).get.column.equalsIgnoreCase("col4"))
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields6") {
@@ -377,7 +378,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields8") {
-    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE-> "col2,col4", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col3")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE-> "col2", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col3")
     val fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     val (dimCols, noDictionary) = stub
@@ -386,29 +387,27 @@ class TestCarbonSqlParser extends QueryTest {
 
     //below dimension fields should be available in dimensions list
     assert(dimCols.size == 7)
-    assert(dimCols.lift(0).get.column.equalsIgnoreCase("col2"))
-    assert(dimCols.lift(1).get.column.equalsIgnoreCase("col3"))
-    assert(dimCols.lift(2).get.column.equalsIgnoreCase("col4"))
+    assert(dimCols.lift(0).get.column.equalsIgnoreCase("col1"))
+    assert(dimCols.lift(1).get.column.equalsIgnoreCase("col2"))
 
     //below column names will be available in noDictionary list
-    assert(noDictionary.size == 2)
+    assert(noDictionary.size == 1)
     assert(noDictionary.lift(0).get.equalsIgnoreCase("col2"))
-    assert(noDictionary.lift(1).get.equalsIgnoreCase("col4"))
 
     //check msr
     assert(msrCols.size == 1)
-    assert(msrCols.lift(0).get.column.equalsIgnoreCase("col1"))
+    assert(msrCols.lift(0).get.column.equalsIgnoreCase("col4"))
   }
 
   // Testing the extracting of measures
   test("Test-extractMsrColsFromFields") {
-    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col2", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col4")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col2", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col1")
     val fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     val msrCols = stub.extractMsrColsFromFieldsTest(fields, tableProperties)
 
     // testing col
-    assert(msrCols.lift(0).get.column.equalsIgnoreCase("col1"))
+    assert(msrCols.lift(0).get.column.equalsIgnoreCase("col4"))
 
   }
 


[27/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-241] delete invalid segments from btree. This closes #158

Posted by ra...@apache.org.
[CARBONDATA-241] delete invalid segments from btree. This closes #158


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

Branch: refs/heads/branch-0.1
Commit: cc5ca8704e3dc6c684a298ad51f74649da604218
Parents: 57ac4a5
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sun Sep 18 02:26:25 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:00:42 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[02/50] [abbrv] incubator-carbondata git commit: Perform equal distribution of dictionary values among the sublists of a list whenever a dictionary file is loaded into memory

Posted by ra...@apache.org.
Perform equal distribution of dictionary values among the sublists of a list whenever a dictionary file is loaded into memory


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

Branch: refs/heads/branch-0.1
Commit: f82ad267ea57321a5bb6cff983bd507270f2bd38
Parents: 2b14960
Author: manishgupt88 <to...@gmail.com>
Authored: Tue Jul 19 14:55:52 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:27:51 2016 +0530

----------------------------------------------------------------------
 .../AbstractColumnDictionaryInfo.java           |  43 +++++---
 .../ColumnDictionaryChunkIterator.java          | 101 +++++++++++++++++++
 .../cache/dictionary/ColumnDictionaryInfo.java  |  47 ++++++++-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  49 +++++++--
 .../core/cache/dictionary/DictionaryInfo.java   |   8 ++
 .../core/reader/CarbonDictionaryReader.java     |   5 +-
 .../core/reader/CarbonDictionaryReaderImpl.java |  43 ++++++--
 .../apache/carbondata/core/util/CarbonUtil.java |  19 ++++
 .../core/writer/CarbonDictionaryWriterImpl.java |  12 +--
 .../writer/CarbonDictionaryWriterImplTest.java  |   6 +-
 10 files changed, 283 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
index a62695c..ad766d7 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/AbstractColumnDictionaryInfo.java
@@ -25,6 +25,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * class that implements cacheable interface and methods specific to column dictionary
@@ -62,6 +63,11 @@ public abstract class AbstractColumnDictionaryInfo implements DictionaryInfo {
   private long dictionaryMetaFileLength;
 
   /**
+   * size of one dictionary bucket
+   */
+  private final int dictionaryOneChunkSize = CarbonUtil.getDictionaryChunkSize();
+
+  /**
    * This method will return the timestamp of file based on which decision
    * the decision will be taken whether to read that file or not
    *
@@ -99,6 +105,16 @@ public abstract class AbstractColumnDictionaryInfo implements DictionaryInfo {
   }
 
   /**
+   * This method will return the size of of last dictionary chunk so that only that many
+   * values are read from the dictionary reader
+   *
+   * @return size of last dictionary chunk
+   */
+  @Override public int getSizeOfLastDictionaryChunk() {
+    return 0;
+  }
+
+  /**
    * This method will decrement the access count for a column by 1
    * whenever a column usage is complete
    */
@@ -241,22 +257,19 @@ public abstract class AbstractColumnDictionaryInfo implements DictionaryInfo {
    */
   protected byte[] getDictionaryBytesFromSurrogate(int surrogateKey) {
     byte[] dictionaryValueInBytes = null;
-    int totalSizeOfDictionaryChunksTraversed = 0;
-    for (List<byte[]> oneDictionaryChunk : dictionaryChunks) {
-      totalSizeOfDictionaryChunksTraversed =
-          totalSizeOfDictionaryChunksTraversed + oneDictionaryChunk.size();
-      // skip the dictionary chunk till surrogate key is lesser than size of
-      // dictionary chunks traversed
-      if (totalSizeOfDictionaryChunksTraversed < surrogateKey) {
-        continue;
+    // surrogate key starts from 1 and list index will start from 0, so lets say if surrogate
+    // key is 10 then value will present at index 9 of the dictionary chunk list
+    int actualSurrogateIndex = surrogateKey - 1;
+    // lets say dictionaryOneChunkSize = 10, surrogateKey = 10, so bucket index will
+    // be 0 and dictionary chunk index will be 9 to get the value
+    int dictionaryBucketIndex = actualSurrogateIndex / dictionaryOneChunkSize;
+    if (dictionaryChunks.size() > dictionaryBucketIndex) {
+      int indexInsideBucket = actualSurrogateIndex % dictionaryOneChunkSize;
+      List<byte[]> dictionaryBucketContainingSurrogateValue =
+          dictionaryChunks.get(dictionaryBucketIndex);
+      if (dictionaryBucketContainingSurrogateValue.size() > indexInsideBucket) {
+        dictionaryValueInBytes = dictionaryBucketContainingSurrogateValue.get(indexInsideBucket);
       }
-      // lets say surrogateKey = 26, total size traversed is 28, dictionary chunk size = 12
-      // then surrogate position in dictionary chunk list is = 26 - (28-12) - 1 = 9
-      // -1 because list index starts from 0
-      int surrogatePositionInDictionaryChunk =
-          surrogateKey - (totalSizeOfDictionaryChunksTraversed - oneDictionaryChunk.size()) - 1;
-      dictionaryValueInBytes = oneDictionaryChunk.get(surrogatePositionInDictionaryChunk);
-      break;
     }
     return dictionaryValueInBytes;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java
new file mode 100644
index 0000000..e87c146
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryChunkIterator.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.core.cache.dictionary;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.apache.carbondata.common.CarbonIterator;
+import org.apache.carbondata.format.ColumnDictionaryChunk;
+
+/**
+ * This class is a wrapper over column dictionary chunk thrift object.
+ * The wrapper class wraps the list<ColumnDictionaryChunk> and provides an API
+ * to fill the byte array into list
+ */
+public class ColumnDictionaryChunkIterator extends CarbonIterator {
+
+  /**
+   * list of dictionaryChunks
+   */
+  private List<ColumnDictionaryChunk> columnDictionaryChunks;
+
+  /**
+   * size of the list
+   */
+  private int size;
+
+  /**
+   * Current index of the list
+   */
+  private int currentSize;
+
+  /**
+   * variable holds the count of elements already iterated
+   */
+  private int iteratorIndex;
+
+  /**
+   * variable holds the current index of List<List<byte[]>> being traversed
+   */
+  private int outerIndex;
+
+  /**
+   * Constructor of ColumnDictionaryChunkIterator
+   *
+   * @param columnDictionaryChunks
+   */
+  public ColumnDictionaryChunkIterator(List<ColumnDictionaryChunk> columnDictionaryChunks) {
+    this.columnDictionaryChunks = columnDictionaryChunks;
+    for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
+      this.size += dictionaryChunk.getValues().size();
+    }
+  }
+
+  /**
+   * Returns {@code true} if the iteration has more elements.
+   * (In other words, returns {@code true} if {@link #next} would
+   * return an element rather than throwing an exception.)
+   *
+   * @return {@code true} if the iteration has more elements
+   */
+  @Override public boolean hasNext() {
+    return (currentSize < size);
+  }
+
+  /**
+   * Returns the next element in the iteration.
+   * The method pics the next elements from the first inner list till first is not finished, pics
+   * the second inner list ...
+   *
+   * @return the next element in the iteration
+   */
+  @Override public byte[] next() {
+    if (iteratorIndex >= columnDictionaryChunks.get(outerIndex).getValues().size()) {
+      iteratorIndex = 0;
+      outerIndex++;
+    }
+    ByteBuffer buffer = columnDictionaryChunks.get(outerIndex).getValues().get(iteratorIndex);
+    byte[] value = buffer.array();
+    currentSize++;
+    iteratorIndex++;
+    return value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
index 08d9bef..b2d81da 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
@@ -30,6 +30,7 @@ import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.ByteUtil;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * class that implements methods specific for dictionary data look up
@@ -112,10 +113,50 @@ public class ColumnDictionaryInfo extends AbstractColumnDictionaryInfo {
   /**
    * This method will add a new dictionary chunk to existing list of dictionary chunks
    *
-   * @param dictionaryChunk
+   * @param newDictionaryChunk
    */
-  @Override public void addDictionaryChunk(List<byte[]> dictionaryChunk) {
-    dictionaryChunks.add(dictionaryChunk);
+  @Override public void addDictionaryChunk(List<byte[]> newDictionaryChunk) {
+    if (dictionaryChunks.size() > 0) {
+      // Ensure that each time a new dictionary chunk is getting added to the
+      // dictionary chunks list, equal distribution of dictionary values should
+      // be there in the sublists of dictionary chunk list
+      List<byte[]> lastDictionaryChunk = dictionaryChunks.get(dictionaryChunks.size() - 1);
+      int dictionaryOneChunkSize = CarbonUtil.getDictionaryChunkSize();
+      int differenceInLastDictionaryAndOneChunkSize =
+          dictionaryOneChunkSize - lastDictionaryChunk.size();
+      if (differenceInLastDictionaryAndOneChunkSize > 0) {
+        // if difference is greater than new dictionary size then copy a part of list
+        // else copy the complete new dictionary chunk list in the last dictionary chunk list
+        if (differenceInLastDictionaryAndOneChunkSize >= newDictionaryChunk.size()) {
+          lastDictionaryChunk.addAll(newDictionaryChunk);
+        } else {
+          List<byte[]> subListOfNewDictionaryChunk =
+              newDictionaryChunk.subList(0, differenceInLastDictionaryAndOneChunkSize);
+          lastDictionaryChunk.addAll(subListOfNewDictionaryChunk);
+          List<byte[]> remainingNewDictionaryChunk = newDictionaryChunk
+              .subList(differenceInLastDictionaryAndOneChunkSize, newDictionaryChunk.size());
+          dictionaryChunks.add(remainingNewDictionaryChunk);
+        }
+      } else {
+        dictionaryChunks.add(newDictionaryChunk);
+      }
+    } else {
+      dictionaryChunks.add(newDictionaryChunk);
+    }
+  }
+
+  /**
+   * This method will return the size of of last dictionary chunk so that only that many
+   * values are read from the dictionary reader
+   *
+   * @return size of last dictionary chunk
+   */
+  @Override public int getSizeOfLastDictionaryChunk() {
+    int lastDictionaryChunkSize = 0;
+    if (dictionaryChunks.size() > 0) {
+      lastDictionaryChunkSize = dictionaryChunks.get(dictionaryChunks.size() - 1).size();
+    }
+    return lastDictionaryChunkSize;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
index 6e603f9..e798c01 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
@@ -20,14 +20,18 @@
 package org.apache.carbondata.core.cache.dictionary;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.common.factory.CarbonCommonFactory;
 import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
 import org.apache.carbondata.core.carbon.ColumnIdentifier;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.reader.CarbonDictionaryReader;
 import org.apache.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
 import org.apache.carbondata.core.service.DictionaryService;
+import org.apache.carbondata.core.util.CarbonUtil;
 
 /**
  * This class is responsible for loading the dictionary data for given columns
@@ -71,12 +75,44 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
   @Override public void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
       long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
       throws IOException {
-    List<byte[]> dictionaryChunk =
+    Iterator<byte[]> columnDictionaryChunkWrapper =
         load(columnIdentifier, dictionaryChunkStartOffset, dictionaryChunkEndOffset);
     if (loadSortIndex) {
       readSortIndexFile(dictionaryInfo, columnIdentifier);
     }
-    dictionaryInfo.addDictionaryChunk(dictionaryChunk);
+    fillDictionaryValuesAndAddToDictionaryChunks(dictionaryInfo, columnDictionaryChunkWrapper);
+  }
+
+  /**
+   * This method will fill the dictionary values according to dictionary bucket size and
+   * add to the dictionary chunk list
+   *
+   * @param dictionaryInfo
+   * @param columnDictionaryChunkWrapper
+   */
+  private void fillDictionaryValuesAndAddToDictionaryChunks(DictionaryInfo dictionaryInfo,
+      Iterator<byte[]> columnDictionaryChunkWrapper) {
+    int dictionaryChunkSize = CarbonUtil.getDictionaryChunkSize();
+    int sizeOfLastDictionaryChunk = dictionaryInfo.getSizeOfLastDictionaryChunk();
+    int sizeOfOneDictionaryChunk = dictionaryChunkSize - sizeOfLastDictionaryChunk;
+    if (sizeOfOneDictionaryChunk == 0) {
+      sizeOfOneDictionaryChunk = dictionaryChunkSize;
+    }
+    List<List<byte[]>> dictionaryChunks =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    List<byte[]> oneDictionaryChunk = new ArrayList<>(sizeOfOneDictionaryChunk);
+    dictionaryChunks.add(oneDictionaryChunk);
+    while (columnDictionaryChunkWrapper.hasNext()) {
+      oneDictionaryChunk.add(columnDictionaryChunkWrapper.next());
+      if (oneDictionaryChunk.size() >= sizeOfOneDictionaryChunk) {
+        sizeOfOneDictionaryChunk = dictionaryChunkSize;
+        oneDictionaryChunk = new ArrayList<>(sizeOfOneDictionaryChunk);
+        dictionaryChunks.add(oneDictionaryChunk);
+      }
+    }
+    for (List<byte[]> dictionaryChunk : dictionaryChunks) {
+      dictionaryInfo.addDictionaryChunk(dictionaryChunk);
+    }
   }
 
   /**
@@ -85,19 +121,18 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    * @param columnIdentifier column unique identifier
    * @param startOffset      start offset of dictionary file
    * @param endOffset        end offset of dictionary file
-   * @return list of dictionary value
+   * @return iterator over dictionary values
    * @throws IOException
    */
-  private List<byte[]> load(ColumnIdentifier columnIdentifier, long startOffset, long endOffset)
+  private Iterator<byte[]> load(ColumnIdentifier columnIdentifier, long startOffset, long endOffset)
       throws IOException {
     CarbonDictionaryReader dictionaryReader = getDictionaryReader(columnIdentifier);
-    List<byte[]> dictionaryValue = null;
     try {
-      dictionaryValue = dictionaryReader.read(startOffset, endOffset);
+      Iterator<byte[]> columnDictionaryChunkWrapper = dictionaryReader.read(startOffset, endOffset);
+      return columnDictionaryChunkWrapper;
     } finally {
       dictionaryReader.close();
     }
-    return dictionaryValue;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
index e34860a..a191545 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/DictionaryInfo.java
@@ -58,6 +58,14 @@ public interface DictionaryInfo extends Cacheable, Dictionary {
   void addDictionaryChunk(List<byte[]> dictionaryChunk);
 
   /**
+   * This method will return the size of of last dictionary chunk so that only that many
+   * values are read from the dictionary reader
+   *
+   * @return size of last dictionary chunk
+   */
+  int getSizeOfLastDictionaryChunk();
+
+  /**
    * This method will set the sort order index of a dictionary column.
    * Sort order index if the index of dictionary values after they are sorted.
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java
index dded6c2..a2289bd 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReader.java
@@ -21,6 +21,7 @@ package org.apache.carbondata.core.reader;
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.List;
 
 /**
@@ -63,8 +64,8 @@ public interface CarbonDictionaryReader extends Closeable {
    *
    * @param startOffset start offset of dictionary file
    * @param endOffset   end offset of dictionary file
-   * @return list of byte array. Each byte array is unique dictionary value
+   * @return iterator over byte array. Each byte array is unique dictionary value
    * @throws IOException if an I/O error occurs
    */
-  List<byte[]> read(long startOffset, long endOffset) throws IOException;
+  Iterator<byte[]> read(long startOffset, long endOffset) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
index a843701..d3f9f31 100644
--- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonDictionaryReaderImpl.java
@@ -22,13 +22,14 @@ package org.apache.carbondata.core.reader;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 
 import org.apache.carbondata.common.factory.CarbonCommonFactory;
+import org.apache.carbondata.core.cache.dictionary.ColumnDictionaryChunkIterator;
 import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
 import org.apache.carbondata.core.carbon.ColumnIdentifier;
 import org.apache.carbondata.core.carbon.path.CarbonTablePath;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.service.PathService;
 import org.apache.carbondata.format.ColumnDictionaryChunk;
 
@@ -115,7 +116,9 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
         carbonDictionaryColumnMetaChunks.get(carbonDictionaryColumnMetaChunks.size() - 1);
     // end offset till where the dictionary file has to be read
     long endOffset = carbonDictionaryColumnMetaChunk.getEnd_offset();
-    return read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
+    List<ColumnDictionaryChunk> columnDictionaryChunks =
+        read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
+    return getDictionaryList(columnDictionaryChunks);
   }
 
   /**
@@ -126,13 +129,17 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    *
    * @param startOffset start offset of dictionary file
    * @param endOffset   end offset of dictionary file
-   * @return list of byte array. Each byte array is unique dictionary value
+   * @return iterator over byte array. Each byte array is unique dictionary value
    * @throws IOException if an I/O error occurs
    */
-  @Override public List<byte[]> read(long startOffset, long endOffset) throws IOException {
+  @Override public Iterator<byte[]> read(long startOffset, long endOffset) throws IOException {
     List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks =
         readDictionaryMetadataFile();
-    return read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
+    List<ColumnDictionaryChunk> columnDictionaryChunks =
+        read(carbonDictionaryColumnMetaChunks, startOffset, endOffset);
+    Iterator<byte[]> columnDictionaryChunkWrapper =
+        new ColumnDictionaryChunkIterator(columnDictionaryChunks);
+    return columnDictionaryChunkWrapper;
   }
 
   /**
@@ -154,11 +161,12 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    * @param startOffset                      start offset for dictionary data file
    * @param endOffset                        end offset till where data has
    *                                         to be read from dictionary data file
-   * @return list of byte array dictionary values
+   * @return list of byte column dictionary values
    * @throws IOException readDictionary file method throws IO exception
    */
-  private List<byte[]> read(List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks,
-      long startOffset, long endOffset) throws IOException {
+  private List<ColumnDictionaryChunk> read(
+      List<CarbonDictionaryColumnMetaChunk> carbonDictionaryColumnMetaChunks, long startOffset,
+      long endOffset) throws IOException {
     // calculate the number of chunks to be read from dictionary file from start offset
     int dictionaryChunkCountsToBeRead =
         calculateTotalDictionaryChunkCountsToBeRead(carbonDictionaryColumnMetaChunks, startOffset,
@@ -168,9 +176,22 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
     // read the required number of chunks from dictionary file
     List<ColumnDictionaryChunk> columnDictionaryChunks =
         readDictionaryFile(startOffset, dictionaryChunkCountsToBeRead);
-    // convert byte buffer list to byte array list of dictionary vlaues
-    List<byte[]> dictionaryValues =
-        new ArrayList<byte[]>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    return columnDictionaryChunks;
+  }
+
+  /**
+   * This method will put all the dictionary chunks into one list and return that list
+   *
+   * @param columnDictionaryChunks
+   * @return
+   */
+  private List<byte[]> getDictionaryList(List<ColumnDictionaryChunk> columnDictionaryChunks) {
+    int dictionaryListSize = 0;
+    for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
+      dictionaryListSize = dictionaryListSize + dictionaryChunk.getValues().size();
+    }
+    // convert byte buffer list to byte array list of dictionary values
+    List<byte[]> dictionaryValues = new ArrayList<byte[]>(dictionaryListSize);
     for (ColumnDictionaryChunk dictionaryChunk : columnDictionaryChunks) {
       convertAndFillByteBufferListToByteArrayList(dictionaryValues, dictionaryChunk.getValues());
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/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 c028efd..2460f6e 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
@@ -1366,5 +1366,24 @@ public final class CarbonUtil {
     }
   }
 
+  /**
+   * initialize the value of dictionary chunk that can be kept in memory at a time
+   *
+   * @return
+   */
+  public static int getDictionaryChunkSize() {
+    int dictionaryOneChunkSize = 0;
+    try {
+      dictionaryOneChunkSize = Integer.parseInt(CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE,
+              CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT));
+    } catch (NumberFormatException e) {
+      dictionaryOneChunkSize =
+          Integer.parseInt(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT);
+      LOGGER.error("Dictionary chunk size not configured properly. Taking default size "
+          + dictionaryOneChunkSize);
+    }
+    return dictionaryOneChunkSize;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index 2e08610..316832d 100644
--- a/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -38,7 +38,6 @@ import org.apache.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.apache.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
 import org.apache.carbondata.core.service.PathService;
-import org.apache.carbondata.core.util.CarbonProperties;
 import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.format.ColumnDictionaryChunk;
 import org.apache.carbondata.format.ColumnDictionaryChunkMeta;
@@ -263,16 +262,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    * initialize the value of dictionary chunk that can be kept in memory at a time
    */
   private void initDictionaryChunkSize() {
-    try {
-      dictionary_one_chunk_size = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE,
-              CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT));
-    } catch (NumberFormatException e) {
-      dictionary_one_chunk_size =
-          Integer.parseInt(CarbonCommonConstants.DICTIONARY_ONE_CHUNK_SIZE_DEFAULT);
-      LOGGER.error("Dictionary chunk size not configured properly. Taking default size "
-              + dictionary_one_chunk_size);
-    }
+    dictionary_one_chunk_size = CarbonUtil.getDictionaryChunkSize();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f82ad267/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
index 8d7653a..5c80bef 100644
--- a/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
@@ -28,6 +28,7 @@ import java.net.URISyntaxException;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Properties;
 import java.util.UUID;
@@ -457,7 +458,10 @@ public class CarbonDictionaryWriterImplTest {
       if (0 == dictionaryEndOffset) {
         dictionaryValues = dictionaryReader.read(dictionaryStartOffset);
       } else {
-        dictionaryValues = dictionaryReader.read(dictionaryStartOffset, dictionaryEndOffset);
+        Iterator<byte[]> itr = dictionaryReader.read(dictionaryStartOffset, dictionaryEndOffset);
+        while (itr.hasNext()) {
+          dictionaryValues.add(itr.next());
+        }
       }
     } finally {
       dictionaryReader.close();


[32/50] [abbrv] incubator-carbondata git commit: add comment option

Posted by ra...@apache.org.
add comment option

fix code style

fixed testcase

renew some files


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

Branch: refs/heads/branch-0.1
Commit: 868ada5597275e079bbbd92076de9fa2c9485d65
Parents: 9b415d7
Author: lion-x <xl...@gmail.com>
Authored: Thu Sep 1 19:55:43 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:08:36 2016 +0530

----------------------------------------------------------------------
 .../hadoop/test/util/StoreCreator.java          |  4 +++
 .../carbondata/spark/load/CarbonLoadModel.java  | 30 ++++++++++++++++++++
 .../carbondata/spark/load/CarbonLoaderUtil.java |  4 +++
 .../spark/util/GlobalDictionaryUtil.scala       |  9 ++++++
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  2 +-
 .../execution/command/carbonTableSchema.scala   |  3 ++
 .../spark/src/test/resources/comment.csv        |  5 ++++
 .../dataload/TestLoadDataWithHiveSyntax.scala   | 20 +++++++++++--
 .../util/ExternalColumnDictionaryTestCase.scala |  1 +
 ...GlobalDictionaryUtilConcurrentTestCase.scala |  1 +
 .../util/GlobalDictionaryUtilTestCase.scala     |  1 +
 .../api/dataloader/DataLoadModel.java           | 13 +++++++++
 .../processing/csvreaderstep/CsvInput.java      |  2 ++
 .../processing/csvreaderstep/CsvInputMeta.java  | 30 ++++++++++++++++++++
 .../csvreaderstep/UnivocityCsvParser.java       |  3 ++
 .../csvreaderstep/UnivocityCsvParserVo.java     | 21 ++++++++++++++
 .../dataprocessor/DataProcessTaskStatus.java    | 14 +++++++++
 .../dataprocessor/IDataProcessStatus.java       |  4 +++
 .../graphgenerator/GraphGenerator.java          |  6 ++++
 19 files changed, 170 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/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 a48e6ad..5867160 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
@@ -353,6 +353,8 @@ public class StoreCreator {
     GraphGenerator.blockInfo.put("qwqwq", new BlockDetails[] { blockDetails });
     schmaModel.setBlocksID("qwqwq");
     schmaModel.setEscapeCharacter("\\");
+    schmaModel.setQuoteCharacter("\"");
+    schmaModel.setCommentCharacter("#");
     info.setDatabaseName(databaseName);
     info.setTableName(tableName);
 
@@ -460,6 +462,8 @@ public class StoreCreator {
     model.setBlocksID(schmaModel.getBlocksID());
     model.setFactTimeStamp(readCurrentTime());
     model.setEscapeCharacter(schmaModel.getEscapeCharacter());
+    model.setQuoteCharacter(schmaModel.getQuoteCharacter());
+    model.setCommentCharacter(schmaModel.getCommentCharacter());
     if (null != loadMetadataDetails && !loadMetadataDetails.isEmpty()) {
       model.setLoadNames(
           CarbonDataProcessorUtil.getLoadNameFromLoadMetaDataDetails(loadMetadataDetails));

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoadModel.java b/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoadModel.java
index c1a073d..68f3929 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoadModel.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoadModel.java
@@ -97,6 +97,16 @@ public class CarbonLoadModel implements Serializable {
   private String escapeChar;
 
   /**
+   * quote Char
+   */
+  private String quoteChar;
+
+  /**
+   * comment Char
+   */
+  private String commentChar;
+
+  /**
    * defines the string that should be treated as null while loadind data
    */
   private String serializationNullFormat;
@@ -322,6 +332,8 @@ public class CarbonLoadModel implements Serializable {
     copy.segmentId = segmentId;
     copy.serializationNullFormat = serializationNullFormat;
     copy.escapeChar = escapeChar;
+    copy.quoteChar = quoteChar;
+    copy.commentChar = commentChar;
     copy.maxColumns = maxColumns;
     return copy;
   }
@@ -361,6 +373,8 @@ public class CarbonLoadModel implements Serializable {
     copyObj.segmentId = segmentId;
     copyObj.serializationNullFormat = serializationNullFormat;
     copyObj.escapeChar = escapeChar;
+    copyObj.quoteChar = quoteChar;
+    copyObj.commentChar = commentChar;
     copyObj.maxColumns = maxColumns;
     return copyObj;
   }
@@ -531,6 +545,22 @@ public class CarbonLoadModel implements Serializable {
     this.serializationNullFormat = serializationNullFormat;
   }
 
+  public String getQuoteChar() {
+    return quoteChar;
+  }
+
+  public void setQuoteChar(String quoteChar) {
+    this.quoteChar = quoteChar;
+  }
+
+  public String getCommentChar() {
+    return commentChar;
+  }
+
+  public void setCommentChar(String commentChar) {
+    this.commentChar = commentChar;
+  }
+
   /**
    * @return
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java b/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
index 246597c..a8302c0 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/load/CarbonLoaderUtil.java
@@ -126,6 +126,8 @@ public final class CarbonLoaderUtil {
     }
     model.setBlocksID(schmaModel.getBlocksID());
     model.setEscapeCharacter(schmaModel.getEscapeCharacter());
+    model.setQuoteCharacter(schmaModel.getQuoteCharacter());
+    model.setCommentCharacter(schmaModel.getCommentCharacter());
     model.setTaskNo(loadModel.getTaskNo());
     model.setFactTimeStamp(loadModel.getFactTimeStamp());
     model.setMaxColumns(loadModel.getMaxColumns());
@@ -180,6 +182,8 @@ public final class CarbonLoaderUtil {
 
     schmaModel.setBlocksID(loadModel.getBlocksID());
     schmaModel.setEscapeCharacter(loadModel.getEscapeChar());
+    schmaModel.setQuoteCharacter(loadModel.getQuoteChar());
+    schmaModel.setCommentCharacter(loadModel.getCommentChar());
     SchemaInfo info = new SchemaInfo();
 
     info.setDatabaseName(databaseName);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index b96a826..7e60320 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -365,6 +365,15 @@ object GlobalDictionaryUtil extends Logging {
       .option("ignoreLeadingWhiteSpace", "false")
       .option("ignoreTrailingWhiteSpace", "false")
       .option("codec", "gzip")
+      .option("quote", {
+        if (StringUtils.isEmpty(carbonLoadModel.getQuoteChar)) {
+          "" + CSVWriter. DEFAULT_QUOTE_CHARACTER
+        }
+        else {
+          carbonLoadModel.getQuoteChar
+        }
+      })
+      .option("comment", carbonLoadModel.getCommentChar)
       .load(carbonLoadModel.getFactFilePath)
     df
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index 69d921f..5675603 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -997,7 +997,7 @@ class CarbonSqlParser()
     val supportedOptions = Seq("DELIMITER", "QUOTECHAR", "FILEHEADER", "ESCAPECHAR", "MULTILINE",
       "COMPLEX_DELIMITER_LEVEL_1", "COMPLEX_DELIMITER_LEVEL_2", "COLUMNDICT",
       "SERIALIZATION_NULL_FORMAT",
-      "ALL_DICTIONARY_PATH", "MAXCOLUMNS"
+      "ALL_DICTIONARY_PATH", "MAXCOLUMNS", "COMMENTCHAR"
     )
     var isSupported = true
     val invalidOptions = StringBuilder.newBuilder

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/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 1e06165..2047872 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
@@ -1091,6 +1091,7 @@ private[sql] case class LoadTable(
       val quoteChar = partionValues.getOrElse("quotechar", "\"")
       val fileHeader = partionValues.getOrElse("fileheader", "")
       val escapeChar = partionValues.getOrElse("escapechar", "\\")
+      val commentchar = partionValues.getOrElse("commentchar", "#")
       val columnDict = partionValues.getOrElse("columndict", null)
       val serializationNullFormat = partionValues.getOrElse("serialization_null_format", "\\N")
       val allDictionaryPath = partionValues.getOrElse("all_dictionary_path", "")
@@ -1107,6 +1108,8 @@ private[sql] case class LoadTable(
       val maxColumns = partionValues.getOrElse("maxcolumns", null)
       carbonLoadModel.setMaxColumns(maxColumns)
       carbonLoadModel.setEscapeChar(escapeChar)
+      carbonLoadModel.setQuoteChar(quoteChar)
+      carbonLoadModel.setCommentChar(commentchar)
       carbonLoadModel.setSerializationNullFormat("serialization_null_format" + "," +
         serializationNullFormat)
       if (delimiter.equalsIgnoreCase(complex_delimiter_level_1) ||

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/test/resources/comment.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/comment.csv b/integration/spark/src/test/resources/comment.csv
new file mode 100644
index 0000000..34ccb12
--- /dev/null
+++ b/integration/spark/src/test/resources/comment.csv
@@ -0,0 +1,5 @@
+.~carbon,.,~carbon,~carbon,~carbon,~carbon,~carbon,~carbon,~carbon
+,carbon,,carbon,,carbon,,carbon,,carbon,,carbon,,carbon,,carbon
+#?carbon,#carbon,#carbon,#carbon,#carbon,#carbon,#carbon,#carbon
+?carbon,#carbon,#carbon,#carbon,#carbon,#carbon,#carbon,#carbon
+".carbon,"carbon,"carbon,"carbon,"carbon,"carbon,"carbon,"carbon
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
index e017a62..071c40c 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
@@ -385,7 +385,7 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
     sql(
       """
        LOAD DATA LOCAL INPATH './src/test/resources/datawithescapecharacter.csv' into table t3
-          options ('DELIMITER'=',', 'QUOTECHAR'='\"','ESCAPECHAR'='\')
+          options ('DELIMITER'=',', 'QUOTECHAR'='"','ESCAPECHAR'='\')
       """
     )
     checkAnswer(sql("select count(*) from t3"), Seq(Row(21)))
@@ -406,7 +406,7 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
     sql(
       """
        LOAD DATA LOCAL INPATH './src/test/resources/datawithescapecharacter.csv' into table t3
-          options ('DELIMITER'=',', 'QUOTECHAR'='\"','ESCAPECHAR'='@')
+          options ('DELIMITER'=',', 'QUOTECHAR'='"','ESCAPECHAR'='@')
       """
     )
     checkAnswer(sql("select count(*) from t3"), Seq(Row(21)))
@@ -590,6 +590,21 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
     checkAnswer(sql("select * from carbontable1"), sql("select * from hivetable1"))
   }
 
+  test("test data loading with comment option") {
+    sql("drop table if exists comment_test")
+    sql(
+      "create table comment_test(imei string, age int, task bigint, num double, level decimal(10," +
+        "3), productdate timestamp, mark int, name string) STORED BY 'org.apache.carbondata.format'"
+    )
+    sql(
+      "LOAD DATA local inpath './src/test/resources/comment.csv' INTO TABLE comment_test " +
+        "options('DELIMITER' = ',', 'QUOTECHAR' = '.', 'COMMENTCHAR' = '?','FILEHEADER'='imei,age,task,num,level,productdate,mark,name')"
+    )
+    checkAnswer(sql("select imei from comment_test"),Seq(Row("\".carbon"),Row("#?carbon"), Row(""),
+      Row("~carbon,")))
+  }
+
+
   override def afterAll {
     sql("drop table carbontable")
     sql("drop table hivetable")
@@ -597,5 +612,6 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists mixed_header_test")
     sql("drop table carbontable1")
     sql("drop table hivetable1")
+    sql("drop table if exists comment_test")
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index 459b99c..4e013bb 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -139,6 +139,7 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
     carbonLoadModel.setComplexDelimiterLevel1("\\$")
     carbonLoadModel.setComplexDelimiterLevel2("\\:")
     carbonLoadModel.setColDictFilePath(extColFilePath)
+    carbonLoadModel.setQuoteChar("\"");
     carbonLoadModel
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
index 8468090..4108abe 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilConcurrentTestCase.scala
@@ -70,6 +70,7 @@ class GlobalDictionaryUtilConcurrentTestCase extends QueryTest with BeforeAndAft
     carbonLoadModel.setComplexDelimiterLevel1("\\$")
     carbonLoadModel.setComplexDelimiterLevel2("\\:")
     carbonLoadModel.setStorePath(relation.tableMeta.storePath)
+    carbonLoadModel.setQuoteChar("\"")
     carbonLoadModel
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
index 05c2715..32beeee 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
@@ -69,6 +69,7 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
     carbonLoadModel.setComplexDelimiterLevel1("\\$")
     carbonLoadModel.setComplexDelimiterLevel2("\\:")
     carbonLoadModel.setStorePath(relation.tableMeta.storePath)
+    carbonLoadModel.setQuoteChar("\"")
     carbonLoadModel
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java b/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java
index 239457b..42a8382 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/api/dataloader/DataLoadModel.java
@@ -61,6 +61,10 @@ public class DataLoadModel {
 
   private String escapeCharacter;
 
+  private String quoteCharacter;
+
+  private String commentCharacter;
+
   private String maxColumns;
   /**
    * @return Returns the schemaInfo.
@@ -200,6 +204,15 @@ public class DataLoadModel {
     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
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
index ab3dbd8..8db8ed4 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
@@ -455,6 +455,8 @@ public class CsvInput extends BaseStep implements StepInterface {
     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));

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/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
index 2e0dece..c5b801a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInputMeta.java
@@ -99,6 +99,10 @@ public class CsvInputMeta extends BaseStepMeta
 
   private String escapeCharacter;
 
+  private String quoteCharacter;
+
+  private String commentCharacter;
+
   private String maxColumns;
 
   public CsvInputMeta() {
@@ -121,6 +125,8 @@ public class CsvInputMeta extends BaseStepMeta
     blocksID = "";
     partitionID = "";
     escapeCharacter ="\\";
+    quoteCharacter = "\"";
+    commentCharacter = "#";
   }
 
   private void readData(Node stepnode) throws KettleXMLException {
@@ -157,6 +163,8 @@ public class CsvInputMeta extends BaseStepMeta
       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");
       maxColumns = XMLHandler.getTagValue(stepnode, "maxColumns");
       Node fields = XMLHandler.getSubNode(stepnode, getXmlCode("FIELDS"));
       int nrfields = XMLHandler.countNodes(fields, getXmlCode("FIELD"));
@@ -220,6 +228,8 @@ public class CsvInputMeta extends BaseStepMeta
     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("maxColumns", maxColumns));
     retval.append("    ").append(XMLHandler.openTag(getXmlCode("FIELDS"))).append(Const.CR);
     for (int i = 0; i < inputFields.length; i++) {
@@ -273,6 +283,8 @@ public class CsvInputMeta extends BaseStepMeta
       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"));
       maxColumns = rep.getStepAttributeString(idStep, getRepCode("maxColumns"));
       int nrfields = rep.countNrStepAttributes(idStep, getRepCode("FIELD_NAME"));
 
@@ -329,6 +341,10 @@ public class CsvInputMeta extends BaseStepMeta
       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("maxColumns"),
           maxColumns);
       for (int i = 0; i < inputFields.length; i++) {
@@ -623,6 +639,16 @@ public class CsvInputMeta extends BaseStepMeta
     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";
   }
@@ -828,6 +854,10 @@ public class CsvInputMeta extends BaseStepMeta
           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 {
           throw new RuntimeException(
               "Unhandled metadata injection of attribute: " + attr.toString() + " - " + attr

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/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
index 51dc1bc..f72dd5b 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java
@@ -97,6 +97,7 @@ public class UnivocityCsvParser {
   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()));
@@ -104,6 +105,8 @@ public class UnivocityCsvParser {
     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));

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/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
index 623cac3..5383309 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParserVo.java
@@ -71,6 +71,16 @@ public class UnivocityCsvParserVo {
   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;
@@ -187,6 +197,17 @@ public class UnivocityCsvParserVo {
     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
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/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
index bc09f89..a0fb157 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
@@ -87,6 +87,10 @@ public class DataProcessTaskStatus implements IDataProcessStatus, Serializable {
 
   private String escapeCharacter;
 
+  private String quoteCharacter;
+
+  private String commentCharacter;
+
   public DataProcessTaskStatus(String databaseName, String tableName) {
     this.databaseName = databaseName;
     this.tableName = tableName;
@@ -283,4 +287,14 @@ public class DataProcessTaskStatus implements IDataProcessStatus, Serializable {
   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;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/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
index 22cada5..56dfe00 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/dataprocessor/IDataProcessStatus.java
@@ -189,4 +189,8 @@ public interface IDataProcessStatus {
   String getBlocksID();
 
   String getEscapeCharacter();
+
+  String getQuoteCharacter();
+
+  String getCommentCharacter();
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/868ada55/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
index f55c247..b10841f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/graphgenerator/GraphGenerator.java
@@ -185,6 +185,8 @@ public class GraphGenerator {
   private String factStoreLocation;
   private String blocksID;
   private String escapeCharacter;
+  private String quoteCharacter;
+  private String commentCharacter;
   /**
    * task id, each spark task has a unique id
    */
@@ -219,6 +221,8 @@ public class GraphGenerator {
     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.factTimeStamp = dataLoadModel.getFactTimeStamp();
     this.segmentId = segmentId;
     this.escapeCharacter = dataLoadModel.getEscapeCharacter();
@@ -450,6 +454,8 @@ public class GraphGenerator {
     csvInputMeta.setBlocksID(this.blocksID);
     csvInputMeta.setPartitionID(this.partitionID);
     csvInputMeta.setEscapeCharacter(this.escapeCharacter);
+    csvInputMeta.setQuoteCharacter(this.quoteCharacter);
+    csvInputMeta.setCommentCharacter(this.commentCharacter);
     csvDataStep.setDraw(true);
     csvDataStep.setDescription("Read raw data from " + GraphGeneratorConstants.CSV_INPUT);
 


[07/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-200] Add performance statistics logs to record the query time taken by carbon This closes #91

Posted by ra...@apache.org.
[CARBONDATA-200] Add performance statistics logs to record the query time taken by carbon This closes #91


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

Branch: refs/heads/branch-0.1
Commit: e1f34ccb57ba5489212c514b4848e66d121e683e
Parents: 346a869
Author: ravipesala <ra...@gmail.com>
Authored: Thu Sep 1 15:57:29 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:31:27 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[03/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-80] This closes #44

Posted by ra...@apache.org.
[CARBONDATA-80] This closes #44


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

Branch: refs/heads/branch-0.1
Commit: 854b75e4e46b3197acc47717714d2d31212934d9
Parents: f82ad26
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Tue Aug 23 19:32:03 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:28:16 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[21/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-247] limit MAXCOLUMNS value . This closes #162

Posted by ra...@apache.org.
[CARBONDATA-247] limit MAXCOLUMNS value . This closes #162


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

Branch: refs/heads/branch-0.1
Commit: e78106a9ca8d09f18aea0803a68d9479791015a2
Parents: 17f5dbe
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sun Sep 18 01:57:24 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:45:58 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[30/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-250] Filter result is not proper when Double data type values with 0.0 and -0.0 will be used.

Posted by ra...@apache.org.
[CARBONDATA-250] Filter result is not proper when Double data type values with 0.0 and -0.0 will be used.


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

Branch: refs/heads/branch-0.1
Commit: c083264a730506bc44fc6387e378d08cab8cc334
Parents: 8b6429a
Author: sujith71955 <su...@gmail.com>
Authored: Sun Sep 18 04:01:10 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:02:11 2016 +0530

----------------------------------------------------------------------
 .../conditional/EqualToExpression.java          |  3 ++-
 .../carbondata/scan/filter/FilterUtil.java      | 21 +++++++++++++++++++-
 .../test/resources/Test_Data1_Logrithmic.csv    |  3 +++
 .../GrtLtFilterProcessorTestCase.scala          | 11 ++++++++++
 4 files changed, 36 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c083264a/core/src/main/java/org/apache/carbondata/scan/expression/conditional/EqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/expression/conditional/EqualToExpression.java b/core/src/main/java/org/apache/carbondata/scan/expression/conditional/EqualToExpression.java
index 12a3e32..8f7fa0a 100644
--- a/core/src/main/java/org/apache/carbondata/scan/expression/conditional/EqualToExpression.java
+++ b/core/src/main/java/org/apache/carbondata/scan/expression/conditional/EqualToExpression.java
@@ -24,6 +24,7 @@ import org.apache.carbondata.scan.expression.Expression;
 import org.apache.carbondata.scan.expression.ExpressionResult;
 import org.apache.carbondata.scan.expression.exception.FilterIllegalMemberException;
 import org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.scan.filter.FilterUtil;
 import org.apache.carbondata.scan.filter.intf.ExpressionType;
 import org.apache.carbondata.scan.filter.intf.RowIntf;
 
@@ -78,7 +79,7 @@ public class EqualToExpression extends BinaryConditionalExpression {
         result = val1.getInt().equals(val2.getInt());
         break;
       case DOUBLE:
-        result = val1.getDouble().equals(val2.getDouble());
+        result = FilterUtil.nanSafeEqualsDoubles(val1.getDouble(), val2.getDouble());
         break;
       case TIMESTAMP:
         result = val1.getTime().equals(val2.getTime());

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c083264a/core/src/main/java/org/apache/carbondata/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/filter/FilterUtil.java b/core/src/main/java/org/apache/carbondata/scan/filter/FilterUtil.java
index 71ac1bf..b7cacb1 100644
--- a/core/src/main/java/org/apache/carbondata/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/apache/carbondata/scan/filter/FilterUtil.java
@@ -1390,6 +1390,26 @@ public final class FilterUtil {
   }
 
   /**
+   * This method will compare double values for its equality and also it will preserve
+   * the -0.0 and 0.0 equality as per == ,also preserve NaN equality check as per
+   * java.lang.Double.equals()
+   *
+   * @param d1 double value for equality check
+   * @param d2 double value for equality check
+   * @return boolean after comparing two double values.
+   */
+  public static boolean nanSafeEqualsDoubles(Double d1, Double d2) {
+    Boolean xIsNan = Double.isNaN(d1);
+    Boolean yIsNan = Double.isNaN(d2);
+    if ((xIsNan && yIsNan) || (d1.doubleValue() == d2.doubleValue())) {
+
+      return true;
+    }
+    return false;
+
+  }
+
+  /**
    * This method will prepare the list with all unknown expressions
    *
    * @param expression
@@ -1406,5 +1426,4 @@ public final class FilterUtil {
       getUnknownExpressionsList(child, lst);
     }
   }
-
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c083264a/integration/spark/src/test/resources/Test_Data1_Logrithmic.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/Test_Data1_Logrithmic.csv b/integration/spark/src/test/resources/Test_Data1_Logrithmic.csv
new file mode 100644
index 0000000..0f0312d
--- /dev/null
+++ b/integration/spark/src/test/resources/Test_Data1_Logrithmic.csv
@@ -0,0 +1,3 @@
+c1_int,c2_Bigint,c3_Decimal,c4_double,c5_string,c6_Timestamp,c7_Datatype_Desc
+2147483646,9223372036854775807,0.12345678900987654321123456789012345638,1.7976931348623157E308,SQL Capability Carbon is fully compliant with Spark SQL and supports all SQL queries which can run directly on Spark SQL Easy Table Definition: Carbon supports easy to use DDL Data Definition Language statements to define and create tables. Carbon DDL is highly flexible and is very easy to use as well as powerful enough to define complex tables Easy Data anagement  Carbon supports a variety of data management functions for loading data to table and maintaining the data in table. Carbon supports bulkloading historical data as well as incrementally loading new data. Loaded data can be deleted based on load time or a specific load can be undone Carbon file format is a columnar store in HDFS  it has many features that a modern columnar format hasz such as splittablez compression schema zcomplex data type and so on. Carbon has following unique features Stores data along with index: it can signi
 ficantly accelerate query performance and reduces the scans and CPU resources where there are filters in the query. Carbon index consists of multiple level of indicesz a processing framework can leverage this index to reduce the task it needs to schedule and processz and it can also do skip scan in more finer grain unit in task side scanning instead of scanning the whole file.Operable encoded data Through supporting efficient compression and global encoding schemes  can query on compressed encoded dataz the data can be converted to encoded data just before returning the results to the usersz which is Column group Allow multiple columns to form a column group that would be stored as row format. This reduces the row reconstruction cost at query time Supports for various use cases with one single Data format  like interactive OLAP-style queryz Sequential Access SQL Capability Carbon is fully compliant with Spark SQL and supports all SQL queries which can run directly on Spark SQL Easy 
 Table Definition: Carbon supports easy to use DDL Data Definition Language statements to define and create tables. Carbon DDL is highly flexible and is very easy to use as well as powerful enough to define complex tables Easy Data anagement  Carbon supports a variety of data management functions for loading data to table and maintaining the data in table. Carbon supports bulkloading historical data as well as incrementally loading new data. Loaded data can be deleted based on load time or a specific load can be undone  it can significantly accelerate query performance and reduces the scans and CPU resources where there are filters in the query. Carbon index consists of multiple level of indicesz a processing framework can leverage this index to reduce the task it needs to schedule and processz and it can also do skip scan in more finer grain unit in task side scanning instead of scanning the whole file Operable encoded data Through supporting efficient compression and global encodin
 g schemes  can query on compressed encoded dataz the data can be converted to encoded data just before returning the results to the usersz which is Column group Allow multiple columns to form a column group that would be stored as row format This reduces the row reconstruction cost at query time Supports for various use cases with one single  Sequential Access Carbon has following unique features Stores data along with index: it can significantly accelerate query performance and reduces the scans and CPU resources where there are filters in the query. Carbon index consists of multiple level of indicesz a processing framework can leverage this index to reduce the task it needs to schedule and processz and it can also do skip scan in more finer grain unit in task side scanning instead of scanning the whole file Operable encoded data Through supporting efficient compression and global encoding schemes  can query on compressed encoded dataz the data can be converted to encoded data just
  before returning the results to the users which is Column group Allow multiple columns to form a column group that would be stored as row format Allow multiple columns to form format Allow carbon is fu,2017-07-01 12:07:28,Max_range_values-1
+2147483646,9223372036854775807,12345678900987654321123456789012345678,1.7976931348623157E308,SQL Capability Carbon is fully compliant with Spark SQL and supports all SQL queries which can run directly on Spark SQL Easy Table Definition: Carbon supports easy to use DDL Data Definition Language statements to define and create tables. Carbon DDL is highly flexible and is very easy to use as well as powerful enough to define complex tables Easy Data anagement  Carbon supports a variety of data management functions for loading data to table and maintaining the data in table. Carbon supports bulkloading historical data as well as incrementally loading new data. Loaded data can be deleted based on load time or a specific load can be undone Carbon file format is a columnar store in HDFS  it has many features that a modern columnar format hasz such as splittablez compression schema zcomplex data type and so on. Carbon has following unique features Stores data along with index: it can signifi
 cantly accelerate query performance and reduces the scans and CPU resources where there are filters in the query. Carbon index consists of multiple level of indicesz a processing framework can leverage this index to reduce the task it needs to schedule and processz and it can also do skip scan in more finer grain unit in task side scanning instead of scanning the whole file.Operable encoded data Through supporting efficient compression and global encoding schemes  can query on compressed encoded dataz the data can be converted to encoded data just before returning the results to the usersz which is Column group Allow multiple columns to form a column group that would be stored as row format. This reduces the row reconstruction cost at query time Supports for various use cases with one single Data format  like interactive OLAP-style queryz Sequential Access SQL Capability Carbon is fully compliant with Spark SQL and supports all SQL queries which can run directly on Spark SQL Easy Ta
 ble Definition: Carbon supports easy to use DDL Data Definition Language statements to define and create tables. Carbon DDL is highly flexible and is very easy to use as well as powerful enough to define complex tables Easy Data anagement  Carbon supports a variety of data management functions for loading data to table and maintaining the data in table. Carbon supports bulkloading historical data as well as incrementally loading new data. Loaded data can be deleted based on load time or a specific load can be undone  it can significantly accelerate query performance and reduces the scans and CPU resources where there are filters in the query. Carbon index consists of multiple level of indicesz a processing framework can leverage this index to reduce the task it needs to schedule and processz and it can also do skip scan in more finer grain unit in task side scanning instead of scanning the whole file Operable encoded data Through supporting efficient compression and global encoding 
 schemes  can query on compressed encoded dataz the data can be converted to encoded data just before returning the results to the usersz which is Column group Allow multiple columns to form a column group that would be stored as row format This reduces the row reconstruction cost at query time Supports for various use cases with one single  Sequential Access Carbon has following unique features Stores data along with index: it can significantly accelerate query performance and reduces the scans and CPU resources where there are filters in the query. Carbon index consists of multiple level of indicesz a processing framework can leverage this index to reduce the task it needs to schedule and processz and it can also do skip scan in more finer grain unit in task side scanning instead of scanning the whole file Operable encoded data Through supporting efficient compression and global encoding schemes  can query on compressed encoded dataz the data can be converted to encoded data just b
 efore returning the results to the users which is Column group Allow multiple columns to form a column group that would be stored as row format Allow multiple columns to form format Allow carbon is f,2017-07-01 12:07:28,Max_range_values-2

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c083264a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
index 5278344..b33b65f 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
@@ -38,6 +38,7 @@ class GrtLtFilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists a12")
     sql("drop table if exists a12_allnull")
     sql("drop table if exists a12_no_null")
+     sql("drop table if exists Test_Boundary1")
 
     sql(
       "create table a12(empid String,ename String,sal double,deptno int,mgr string,gender string," +
@@ -53,6 +54,7 @@ class GrtLtFilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
         " string," +
         "dob timestamp,comm decimal(4,2),desc string) stored by 'org.apache.carbondata.format'"
     )
+    sql("create table Test_Boundary1 (c1_int int,c2_Bigint Bigint,c3_Decimal Decimal(38,38),c4_double double,c5_string string,c6_Timestamp Timestamp,c7_Datatype_Desc string) STORED BY 'org.apache.carbondata.format'")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
     val basePath = new File(this.getClass.getResource("/").getPath + "/../../")
@@ -77,6 +79,9 @@ class GrtLtFilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
          'QUOTECHAR'='"')"""
         .stripMargin
     )
+    
+    sql(
+      s"LOAD DATA INPATH './src/test/resources/Test_Data1_Logrithmic.csv' INTO table Test_Boundary1 OPTIONS('DELIMITER'=',','QUOTECHAR'='','FILEHEADER'='')")
   }
   //mixed value test
   test("Less Than Filter") {
@@ -99,6 +104,12 @@ class GrtLtFilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
       Seq(Row(3))
     )
   }
+  test("0.0 and -0.0 equality check for double data type applying log function") {
+    checkAnswer(
+      sql("select log(c4_double,1) from Test_Boundary1 where log(c4_double,1)= -0.0"),
+      Seq(Row(0.0),Row(0.0))
+    )
+  }
 
   test("Greater Than equal to Filter") {
     sql("select count (empid) from a12 where dob >= '2014-07-01 12:07:28'").show()


[18/50] [abbrv] incubator-carbondata git commit: Throwing exception in case of improper MAXCOULUMN value

Posted by ra...@apache.org.
Throwing exception in case of improper MAXCOULUMN value


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

Branch: refs/heads/branch-0.1
Commit: 6861b7e4544fe01fd8a4f3c6d2f812f465c307d2
Parents: 595460c
Author: Manohar <ma...@gmail.com>
Authored: Sat Sep 17 19:17:58 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:44:44 2016 +0530

----------------------------------------------------------------------
 .../TestDataLoadWithColumnsMoreThanSchema.scala | 20 ++++++++++++++++++++
 .../processing/csvload/DataGraphExecuter.java   |  2 +-
 .../processing/csvreaderstep/CsvInput.java      |  1 +
 3 files changed, 22 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6861b7e4/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
index 4755a01..7bd29d5 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
@@ -25,6 +25,8 @@ import org.apache.spark.sql.common.util.CarbonHiveContext._
 import org.apache.spark.sql.common.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
+import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
+
 /**
  * This class will test data load in which number of columns in data are more than
  * the number of columns in schema
@@ -68,8 +70,26 @@ class TestDataLoadWithColumnsMoreThanSchema extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test with invalid maxColumns value") {
+    sql(
+      "CREATE TABLE max_columns_value_test (imei string,age int,task bigint,num double,level " +
+      "decimal(10,3),productdate timestamp,mark int,name string) STORED BY 'org.apache.carbondata" +
+      ".format'")
+    try {
+      sql(
+        "LOAD DATA LOCAL INPATH './src/test/resources/character_carbon.csv' into table " +
+        "max_columns_value_test options('FILEHEADER='imei,age','MAXCOLUMNS'='2')")
+      throw new MalformedCarbonCommandException("Invalid")
+    } catch {
+      case me: MalformedCarbonCommandException =>
+        assert(false)
+      case _ => assert(true)
+    }
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS char_test")
     sql("DROP TABLE IF EXISTS hive_char_test")
+    sql("DROP TABLE IF EXISTS max_columns_value_test")
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6861b7e4/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java b/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
index 67d77a3..680cb37 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvload/DataGraphExecuter.java
@@ -209,7 +209,7 @@ public class DataGraphExecuter {
 
     if (trans.getErrors() > 0) {
       LOGGER.error("Graph Execution had errors");
-      throw new DataLoadingException("Internal Errors");
+      throw new DataLoadingException("Due to internal errors, please check logs for more details.");
     } else if (null != BadRecordslogger.hasBadRecord(key)) {
       LOGGER.error("Graph Execution is partcially success");
       throw new DataLoadingException(DataProcessorConstants.BAD_REC_FOUND,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6861b7e4/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
index c01a682..ab3dbd8 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/CsvInput.java
@@ -390,6 +390,7 @@ public class CsvInput extends BaseStep implements StepInterface {
           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;
       }


[36/50] [abbrv] incubator-carbondata git commit: Handle all dictionary exception more properly

Posted by ra...@apache.org.
Handle all dictionary exception more properly

modifed by suggestions


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

Branch: refs/heads/branch-0.1
Commit: eedfe5969edbcca8ea9ec0bc18651af7320ed831
Parents: b53a4db
Author: foryou2030 <fo...@126.com>
Authored: Sat Aug 27 14:42:16 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:30:16 2016 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  4 +-
 .../store/filesystem/LocalCarbonFile.java       |  5 +-
 .../spark/util/GlobalDictionaryUtil.scala       | 54 ++++++++++++--------
 3 files changed, 39 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eedfe596/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 1e603b5..6050719 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
@@ -199,9 +199,9 @@ public final class CarbonCommonConstants {
    */
   public static final String MEMBER_DEFAULT_VAL = "@NU#LL$!";
   /**
-   * BLANK_LINE_FLAG
+   * DEFAULT_COLUMN_NAME
    */
-  public static final String BLANK_LINE_FLAG = "@NU#LL$!BLANKLINE";
+  public static final String DEFAULT_COLUMN_NAME = "@NU#LL$!COLUMN";
   /**
    * FILE STATUS IN-PROGRESS
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eedfe596/core/src/main/java/org/apache/carbondata/core/datastorage/store/filesystem/LocalCarbonFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/filesystem/LocalCarbonFile.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/filesystem/LocalCarbonFile.java
index f46aeed..406f6d1 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/filesystem/LocalCarbonFile.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/filesystem/LocalCarbonFile.java
@@ -86,7 +86,10 @@ public class LocalCarbonFile implements CarbonFile {
   }
 
   @Override public boolean exists() {
-    return file.exists();
+    if (file != null) {
+      return file.exists();
+    }
+    return false;
   }
 
   @Override public String getCanonicalPath() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/eedfe596/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index cabedfd..bdd8adc 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.spark.util
 
-import java.io.IOException
+import java.io.{FileNotFoundException, IOException}
 import java.nio.charset.Charset
 import java.util.regex.Pattern
 
@@ -602,29 +602,31 @@ object GlobalDictionaryUtil extends Logging {
       val basicRdd = sqlContext.sparkContext.textFile(allDictionaryPath)
         .map(x => {
         val tokens = x.split("" + CSVWriter.DEFAULT_SEPARATOR)
-        var index: Int = 0
+        if (tokens.size != 2) {
+          logError("Read a bad dictionary record: " + x)
+        }
+        var columnName: String = CarbonCommonConstants.DEFAULT_COLUMN_NAME
         var value: String = ""
         try {
-          index = tokens(0).toInt
+          columnName = csvFileColumns(tokens(0).toInt)
           value = tokens(1)
         } catch {
           case ex: Exception =>
-            logError("read a bad dictionary record" + x)
+            logError("Reset bad dictionary record as default value")
         }
-        (index, value)
+        (columnName, value)
       })
+
       // group by column index, and filter required columns
       val requireColumnsList = requireColumns.toList
       allDictionaryRdd = basicRdd
         .groupByKey()
-        .map(x => (csvFileColumns(x._1), x._2))
         .filter(x => requireColumnsList.contains(x._1))
     } catch {
       case ex: Exception =>
-        logError("read local dictionary files failed")
+        logError("Read dictionary files failed. Caused by: " + ex.getMessage)
         throw ex
     }
-
     allDictionaryRdd
   }
 
@@ -640,22 +642,32 @@ object GlobalDictionaryUtil extends Logging {
     // filepath regex, look like "/path/*.dictionary"
     if (filePath.getName.startsWith("*")) {
       val dictExt = filePath.getName.substring(1)
-      val listFiles = filePath.getParentFile.listFiles()
-      if (listFiles.exists(file =>
-        file.getName.endsWith(dictExt) && file.getSize > 0)) {
-        true
+      if (filePath.getParentFile.exists()) {
+        val listFiles = filePath.getParentFile.listFiles()
+        if (listFiles.exists(file =>
+          file.getName.endsWith(dictExt) && file.getSize > 0)) {
+          true
+        } else {
+          logWarning("No dictionary files found or empty dictionary files! " +
+            "Won't generate new dictionary.")
+          false
+        }
       } else {
-        logInfo("No dictionary files found or empty dictionary files! " +
-          "Won't generate new dictionary.")
-        false
+        throw new FileNotFoundException(
+          "The given dictionary file path is not found!")
       }
     } else {
-      if (filePath.exists() && filePath.getSize > 0) {
-        true
+      if (filePath.exists()) {
+        if (filePath.getSize > 0) {
+          true
+        } else {
+          logWarning("No dictionary files found or empty dictionary files! " +
+            "Won't generate new dictionary.")
+          false
+        }
       } else {
-        logInfo("No dictionary files found or empty dictionary files! " +
-          "Won't generate new dictionary.")
-        false
+        throw new FileNotFoundException(
+          "The given dictionary file path is not found!")
       }
     }
   }
@@ -768,7 +780,7 @@ object GlobalDictionaryUtil extends Logging {
           }
         }
       } else {
-        logInfo("Generate global dictionary from all dictionary files!")
+        logInfo("Generate global dictionary from dictionary files!")
         val isNonempty = validateAllDictionaryPath(allDictionaryPath)
         if(isNonempty) {
           var headers = if (StringUtils.isEmpty(carbonLoadModel.getCsvHeader)) {


[22/50] [abbrv] incubator-carbondata git commit: clear query statistics map

Posted by ra...@apache.org.
clear query statistics map


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

Branch: refs/heads/branch-0.1
Commit: 60fde4123514c46883f8d1f131d9c0e518b1e981
Parents: e78106a
Author: foryou2030 <fo...@126.com>
Authored: Fri Sep 2 18:22:03 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:57:11 2016 +0530

----------------------------------------------------------------------
 .../DriverQueryStatisticsRecorder.java          | 234 +++++++++++--------
 .../QueryStatisticsConstants.java               |   3 +
 .../apache/carbondata/core/util/CarbonUtil.java |  14 ++
 3 files changed, 158 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60fde412/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
index 9e83c22..534852f 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
@@ -19,15 +19,19 @@
 package org.apache.carbondata.core.carbon.querystatistics;
 
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 
+import static org.apache.carbondata.core.util.CarbonUtil.add;
 import static org.apache.carbondata.core.util.CarbonUtil.printLine;
 
+import org.apache.commons.lang3.StringUtils;
+
 /**
  * Class will be used to record and log the query statistics
  */
@@ -47,7 +51,8 @@ public class DriverQueryStatisticsRecorder {
   private static final Object lock = new Object();
 
   private DriverQueryStatisticsRecorder() {
-    queryStatisticsMap = new HashMap<String, List<QueryStatistic>>();
+    // use ConcurrentHashMap, it is thread-safe
+    queryStatisticsMap = new ConcurrentHashMap<String, List<QueryStatistic>>();
   }
 
   private static DriverQueryStatisticsRecorder carbonLoadStatisticsImplInstance =
@@ -78,9 +83,32 @@ public class DriverQueryStatisticsRecorder {
    */
   public void logStatisticsAsTableDriver() {
     synchronized (lock) {
-      String tableInfo = collectDriverStatistics();
-      if (null != tableInfo) {
-        LOGGER.statistic(tableInfo);
+      Iterator<Map.Entry<String, List<QueryStatistic>>> entries =
+              queryStatisticsMap.entrySet().iterator();
+      while (entries.hasNext()) {
+        Map.Entry<String, List<QueryStatistic>> entry = entries.next();
+        String queryId = entry.getKey();
+        // clear the unknown query statistics
+        if(StringUtils.isEmpty(queryId)) {
+          entries.remove();
+        } else {
+          // clear the timeout query statistics
+          long interval = System.nanoTime() - Long.parseLong(queryId);
+          if (interval > QueryStatisticsConstants.CLEAR_STATISTICS_TIMEOUT) {
+            entries.remove();
+          } else {
+            // print sql_parse_t,load_meta_t,block_allocation_t,block_identification_t
+            // or just print block_allocation_t,block_identification_t
+            if (entry.getValue().size() >= 2) {
+              String tableInfo = collectDriverStatistics(entry.getValue(), queryId);
+              if (null != tableInfo) {
+                LOGGER.statistic(tableInfo);
+                // clear the statistics that has been printed
+                entries.remove();
+              }
+            }
+          }
+        }
       }
     }
   }
@@ -88,96 +116,116 @@ public class DriverQueryStatisticsRecorder {
   /**
    * Below method will parse queryStatisticsMap and put time into table
    */
-  public String collectDriverStatistics() {
-    for (String key: queryStatisticsMap.keySet()) {
-      try {
-        // TODO: get the finished query, and print Statistics
-        if (queryStatisticsMap.get(key).size() > 3) {
-          String sql_parse_time = "";
-          String load_meta_time = "";
-          String block_allocation_time = "";
-          String block_identification_time = "";
-          Double driver_part_time_tmp = 0.0;
-          String splitChar = " ";
-          // get statistic time from the QueryStatistic
-          for (QueryStatistic statistic : queryStatisticsMap.get(key)) {
-            switch (statistic.getMessage()) {
-              case QueryStatisticsConstants.SQL_PARSE:
-                sql_parse_time += statistic.getTimeTaken() + splitChar;
-                driver_part_time_tmp += statistic.getTimeTaken();
-                break;
-              case QueryStatisticsConstants.LOAD_META:
-                load_meta_time += statistic.getTimeTaken() + splitChar;
-                driver_part_time_tmp += statistic.getTimeTaken();
-                break;
-              case QueryStatisticsConstants.BLOCK_ALLOCATION:
-                block_allocation_time += statistic.getTimeTaken() + splitChar;
-                driver_part_time_tmp += statistic.getTimeTaken();
-                break;
-              case QueryStatisticsConstants.BLOCK_IDENTIFICATION:
-                block_identification_time += statistic.getTimeTaken() + splitChar;
-                driver_part_time_tmp += statistic.getTimeTaken();
-                break;
-              default:
-                break;
-            }
-          }
-          String driver_part_time = driver_part_time_tmp + splitChar;
-          // structure the query statistics info table
-          StringBuilder tableInfo = new StringBuilder();
-          int len1 = 8;
-          int len2 = 20;
-          int len3 = 21;
-          int len4 = 22;
-          String line = "+" + printLine("-", len1) + "+" + printLine("-", len2) + "+" +
-              printLine("-", len3) + "+" + printLine("-", len4) + "+";
-          String line2 = "|" + printLine(" ", len1) + "+" + printLine("-", len2) + "+" +
-              printLine(" ", len3) + "+" + printLine("-", len4) + "+";
-          // table header
-          tableInfo.append(line).append("\n");
-          tableInfo.append("|" + printLine(" ", (len1 - "Module".length())) + "Module" + "|" +
-              printLine(" ", (len2 - "Operation Step".length())) + "Operation Step" + "|" +
-              printLine(" ", (len3 + len4 + 1 - "Query Cost".length())) +
-              "Query Cost" + "|" + "\n");
-          // driver part
-          tableInfo.append(line).append("\n");
-          tableInfo.append("|" + printLine(" ", len1) + "|" +
-              printLine(" ", (len2 - "SQL parse".length())) + "SQL parse" + "|" +
-              printLine(" ", len3) + "|" +
-              printLine(" ", (len4 - sql_parse_time.length())) + sql_parse_time + "|" + "\n");
-          tableInfo.append(line2).append("\n");
-          tableInfo.append("|" +printLine(" ", (len1 - "Driver".length())) + "Driver" + "|" +
-              printLine(" ", (len2 - "Load meta data".length())) + "Load meta data" + "|" +
-              printLine(" ", (len3 - driver_part_time.length())) + driver_part_time + "|" +
-              printLine(" ", (len4 - load_meta_time.length())) +
-              load_meta_time + "|" + "\n");
-          tableInfo.append(line2).append("\n");
-          tableInfo.append("|" +
-              printLine(" ", (len1 - "Part".length())) + "Part" + "|" +
-              printLine(" ", (len2 - "Block allocation".length())) +
-              "Block allocation" + "|" +
-              printLine(" ", len3) + "|" +
-              printLine(" ", (len4 - block_allocation_time.length())) +
-              block_allocation_time + "|" + "\n");
-          tableInfo.append(line2).append("\n");
-          tableInfo.append("|" +
-              printLine(" ", len1) + "|" +
-              printLine(" ", (len2 - "Block identification".length())) +
-              "Block identification" + "|" +
-              printLine(" ", len3) + "|" +
-              printLine(" ", (len4 - block_identification_time.length())) +
-              block_identification_time + "|" + "\n");
-          tableInfo.append(line).append("\n");
-
-          // once the statistics be printed, remove it from the map
-          queryStatisticsMap.remove(key);
-          // show query statistic as "query id" + "table"
-          return "Print query statistic for query id: " + key + "\n" + tableInfo.toString();
+  public String collectDriverStatistics(List<QueryStatistic> statisticsList, String queryId) {
+    String sql_parse_time = "";
+    String load_meta_time = "";
+    String block_allocation_time = "";
+    String block_identification_time = "";
+    Double driver_part_time_tmp = 0.0;
+    Double driver_part_time_tmp2 = 0.0;
+    String splitChar = " ";
+    try {
+      // get statistic time from the QueryStatistic
+      for (QueryStatistic statistic : statisticsList) {
+        switch (statistic.getMessage()) {
+          case QueryStatisticsConstants.SQL_PARSE:
+            sql_parse_time += statistic.getTimeTaken() + splitChar;
+            driver_part_time_tmp = add(driver_part_time_tmp, statistic.getTimeTaken());
+            break;
+          case QueryStatisticsConstants.LOAD_META:
+            load_meta_time += statistic.getTimeTaken() + splitChar;
+            driver_part_time_tmp = add(driver_part_time_tmp, statistic.getTimeTaken());
+            break;
+          case QueryStatisticsConstants.BLOCK_ALLOCATION:
+            block_allocation_time += statistic.getTimeTaken() + splitChar;
+            driver_part_time_tmp = add(driver_part_time_tmp, statistic.getTimeTaken());
+            driver_part_time_tmp2 = add(driver_part_time_tmp2, statistic.getTimeTaken());
+            break;
+          case QueryStatisticsConstants.BLOCK_IDENTIFICATION:
+            block_identification_time += statistic.getTimeTaken() + splitChar;
+            driver_part_time_tmp = add(driver_part_time_tmp, statistic.getTimeTaken());
+            driver_part_time_tmp2 = add(driver_part_time_tmp2, statistic.getTimeTaken());
+            break;
+          default:
+            break;
         }
-      } catch (Exception ex) {
-        return "Put statistics into table failed, catch exception: " + ex.getMessage();
       }
+      String driver_part_time = driver_part_time_tmp + splitChar;
+      // structure the query statistics info table
+      StringBuilder tableInfo = new StringBuilder();
+      int len1 = 8;
+      int len2 = 20;
+      int len3 = 21;
+      int len4 = 22;
+      String line = "+" + printLine("-", len1) + "+" + printLine("-", len2) + "+" +
+          printLine("-", len3) + "+" + printLine("-", len4) + "+";
+      String line2 = "|" + printLine(" ", len1) + "+" + printLine("-", len2) + "+" +
+          printLine(" ", len3) + "+" + printLine("-", len4) + "+";
+      // table header
+      tableInfo.append(line).append("\n");
+      tableInfo.append("|" + printLine(" ", (len1 - "Module".length())) + "Module" + "|" +
+          printLine(" ", (len2 - "Operation Step".length())) + "Operation Step" + "|" +
+          printLine(" ", (len3 + len4 + 1 - "Query Cost".length())) + "Query Cost" + "|" + "\n");
+      tableInfo.append(line).append("\n");
+      // print sql_parse_t,load_meta_t,block_allocation_t,block_identification_t
+      if (!StringUtils.isEmpty(sql_parse_time) &&
+          !StringUtils.isEmpty(load_meta_time) &&
+          !StringUtils.isEmpty(block_allocation_time) &&
+          !StringUtils.isEmpty(block_identification_time)) {
+        tableInfo.append("|" + printLine(" ", len1) + "|" +
+            printLine(" ", (len2 - "SQL parse".length())) + "SQL parse" + "|" +
+            printLine(" ", len3) + "|" +
+            printLine(" ", (len4 - sql_parse_time.length())) + sql_parse_time + "|" + "\n");
+        tableInfo.append(line2).append("\n");
+        tableInfo.append("|" + printLine(" ", (len1 - "Driver".length())) + "Driver" + "|" +
+            printLine(" ", (len2 - "Load meta data".length())) + "Load meta data" + "|" +
+            printLine(" ", (len3 - driver_part_time.length())) + driver_part_time + "|" +
+            printLine(" ", (len4 - load_meta_time.length())) +
+            load_meta_time + "|" + "\n");
+        tableInfo.append(line2).append("\n");
+        tableInfo.append("|" + printLine(" ", (len1 - "Part".length())) + "Part" + "|" +
+            printLine(" ", (len2 - "Block allocation".length())) + "Block allocation" + "|" +
+            printLine(" ", len3) + "|" +
+            printLine(" ", (len4 - block_allocation_time.length())) +
+            block_allocation_time + "|" + "\n");
+        tableInfo.append(line2).append("\n");
+        tableInfo.append("|" +
+            printLine(" ", len1) + "|" +
+            printLine(" ", (len2 - "Block identification".length())) +
+            "Block identification" + "|" +
+            printLine(" ", len3) + "|" +
+            printLine(" ", (len4 - block_identification_time.length())) +
+            block_identification_time + "|" + "\n");
+        tableInfo.append(line).append("\n");
+
+        // show query statistic as "query id" + "table"
+        return "Print query statistic for query id: " + queryId + "\n" + tableInfo.toString();
+      } else if (!StringUtils.isEmpty(block_allocation_time) &&
+          !StringUtils.isEmpty(block_identification_time)) {
+        // when we can't get sql parse time, we only print the last two
+        driver_part_time = driver_part_time_tmp2 + splitChar;
+        tableInfo.append("|" + printLine(" ", (len1 - "Driver".length())) + "Driver" + "|" +
+            printLine(" ", (len2 - "Block allocation".length())) + "Block allocation" + "|" +
+            printLine(" ", (len3 - driver_part_time.length())) + driver_part_time + "|" +
+            printLine(" ", (len4 - block_allocation_time.length())) +
+            block_allocation_time + "|" + "\n");
+        tableInfo.append(line2).append("\n");
+        tableInfo.append("|" +
+            printLine(" ", (len1 - "Part".length())) + "Part" + "|" +
+            printLine(" ", (len2 - "Block identification".length())) +
+            "Block identification" + "|" +
+            printLine(" ", len3) + "|" +
+            printLine(" ", (len4 - block_identification_time.length())) +
+            block_identification_time + "|" + "\n");
+        tableInfo.append(line).append("\n");
+
+        // show query statistic as "query id" + "table"
+        return "Print query statistic for query id: " + queryId + "\n" + tableInfo.toString();
+      }
+
+      return null;
+    } catch (Exception ex) {
+      return "Put statistics into table failed, catch exception: " + ex.getMessage();
     }
-    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60fde412/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java
index 540cf1e..118dadb 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java
@@ -48,6 +48,9 @@ public interface QueryStatisticsConstants {
 
   String RESULT_SIZE = "The size of query result";
 
+  // clear no-use statistics timeout
+  long CLEAR_STATISTICS_TIMEOUT = 60 * 1000 * 1000000L;
+
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60fde412/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 77d041c..c98456e 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
@@ -28,6 +28,7 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
+import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.charset.Charset;
@@ -1430,5 +1431,18 @@ public final class CarbonUtil {
     }
     return builder.toString();
   }
+
+  /**
+   * Below method will for double plus double
+   *
+   * @param v1
+   * @param v2
+   */
+  public static double add(double v1, double v2)
+  {
+    BigDecimal b1 = new BigDecimal(Double.toString(v1));
+    BigDecimal b2 = new BigDecimal(Double.toString(v2));
+    return  b1.add(b2).doubleValue();
+  }
 }
 


[15/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-224] Data mismatch issue in Dictionary Exclude column for Numeric data type. This closes #139

Posted by ra...@apache.org.
[CARBONDATA-224] Data mismatch issue in Dictionary Exclude column for Numeric data type. This closes #139


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

Branch: refs/heads/branch-0.1
Commit: 145c7af119da675995c94f51e2c3e41afdfccd4e
Parents: 7cfc3ec
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Fri Sep 16 20:21:28 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:40:10 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[20/50] [abbrv] incubator-carbondata git commit: Problem: Higher MAXCOLUMNS value in load DML options is leading to out of memory error

Posted by ra...@apache.org.
Problem: Higher MAXCOLUMNS value in load DML options is leading to out of memory error

Analysis: When a higher value lets say Integer max value is configured for maxcolumns option in load DML and executor memory is less, then in that case UnivocityCsvParser throws an out of memory error when it tries to create an array of size of maxColumns option value.

Fix: Set the threshold value for maxColumns option value that our system can support and if maxColumns option value is greater than threshold value then assign the threshold value to maxColumns option value

Impact: Data loading


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

Branch: refs/heads/branch-0.1
Commit: 17f5dbee3972b1934ed9f1246b6bf00759f647aa
Parents: 87ba555
Author: manishgupta88 <to...@gmail.com>
Authored: Sat Sep 17 10:52:27 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:45:35 2016 +0530

----------------------------------------------------------------------
 .../TestDataLoadWithColumnsMoreThanSchema.scala        | 12 ++++++++++++
 .../processing/csvreaderstep/UnivocityCsvParser.java   | 13 +++++++++++--
 2 files changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/17f5dbee/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
index 7bd29d5..4e5a207 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestDataLoadWithColumnsMoreThanSchema.scala
@@ -87,6 +87,18 @@ class TestDataLoadWithColumnsMoreThanSchema extends QueryTest with BeforeAndAfte
     }
   }
 
+  test("test for maxcolumns option value greater than threshold value for maxcolumns") {
+    sql("DROP TABLE IF EXISTS valid_max_columns_test")
+    sql("CREATE TABLE valid_max_columns_test (imei string,age int,task bigint,num double,level decimal(10,3),productdate timestamp,mark int,name string)STORED BY 'org.apache.carbondata.format'")
+    try {
+      sql("LOAD DATA LOCAL INPATH './src/test/resources/character_carbon.csv' into table valid_max_columns_test options('MAXCOLUMNS'='22000')")
+      checkAnswer(sql("select count(*) from valid_max_columns_test"),
+        sql("select count(*) from hive_char_test"))
+    } catch {
+      case _ => assert(false)
+    }
+  }
+
   override def afterAll {
     sql("DROP TABLE IF EXISTS char_test")
     sql("DROP TABLE IF EXISTS hive_char_test")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/17f5dbee/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
index 49f17dc..51dc1bc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/csvreaderstep/UnivocityCsvParser.java
@@ -49,6 +49,10 @@ public class UnivocityCsvParser {
    */
   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;
@@ -122,12 +126,17 @@ public class UnivocityCsvParser {
     int maxNumberOfColumnsForParsing = DEFAULT_MAX_NUMBER_OF_COLUMNS_FOR_PARSING;
     if (maxColumns > 0) {
       if (columnCountInSchema > maxColumns) {
-        maxNumberOfColumnsForParsing = columnCountInSchema + 10;
+        maxNumberOfColumnsForParsing = columnCountInSchema;
+      } 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) {
-      maxNumberOfColumnsForParsing = columnCountInSchema + 10;
+      maxNumberOfColumnsForParsing = columnCountInSchema;
     }
     return maxNumberOfColumnsForParsing;
   }


[44/50] [abbrv] incubator-carbondata git commit: Corrected file headers and removed unnecessary open csv files

Posted by ra...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/ResultSetHelper.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/ResultSetHelper.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/ResultSetHelper.java
deleted file mode 100644
index 195374a..0000000
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/ResultSetHelper.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.carbondata.spark.partition.reader;
-/**
- * Copyright 2005 Bytecode Pty Ltd.
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-
-/**
- * Interface for the ResultSetHelperService.  Allows the user to define their own ResultSetHelper
- * for use in the CSVWriter.
- */
-public interface ResultSetHelper {
-  /**
-   * Returns the column Names from the ResultSet.
-   *
-   * @param rs - ResultSet
-   * @return - string array containing the column names.
-   * @throws SQLException - thrown by the ResultSet.
-   */
-  String[] getColumnNames(ResultSet rs) throws SQLException;
-
-  /**
-   * Returns the column values from the result set.
-   *
-   * @param rs - the ResultSet containing the values.
-   * @return String Array containing the values.
-   * @throws SQLException - thrown by the ResultSet.
-   * @throws IOException  - thrown by the ResultSet.
-   */
-  String[] getColumnValues(ResultSet rs) throws SQLException, IOException;
-
-  /**
-   * Returns the column values from the result set with the values trimmed if desired.
-   *
-   * @param rs   - the ResultSet containing the values.
-   * @param trim - values should have white spaces trimmed.
-   * @return String Array containing the values.
-   * @throws SQLException - thrown by the ResultSet.
-   * @throws IOException  - thrown by the ResultSet.
-   */
-  String[] getColumnValues(ResultSet rs, boolean trim) throws SQLException, IOException;
-
-  /**
-   * Returns the column values from the result set with the values trimmed if desired.
-   * Also format the date and time columns based on the format strings passed in.
-   *
-   * @param rs               - the ResultSet containing the values.
-   * @param trim             - values should have white spaces trimmed.
-   * @param dateFormatString - format String for dates.
-   * @param timeFormatString - format String for timestamps.
-   * @return String Array containing the values.
-   * @throws SQLException - thrown by the ResultSet.
-   * @throws IOException  - thrown by the ResultSet.
-   */
-  String[] getColumnValues(ResultSet rs, boolean trim, String dateFormatString,
-      String timeFormatString) throws SQLException, IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/ResultSetHelperService.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/ResultSetHelperService.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/ResultSetHelperService.java
deleted file mode 100644
index 3d15949..0000000
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/ResultSetHelperService.java
+++ /dev/null
@@ -1,327 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.carbondata.spark.partition.reader;
-/**
- * Copyright 2005 Bytecode Pty Ltd.
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-import java.io.Reader;
-import java.math.BigDecimal;
-import java.sql.Clob;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.sql.Types;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-/**
- * helper class for processing JDBC ResultSet objects.
- */
-public class ResultSetHelperService implements ResultSetHelper {
-  public static final int CLOBBUFFERSIZE = 2048;
-
-  // note: we want to maintain compatibility with Java 5 VM's
-  // These types don't exist in Java 5
-  static final int NVARCHAR = -9;
-  static final int NCHAR = -15;
-  static final int LONGNVARCHAR = -16;
-  static final int NCLOB = 2011;
-
-  static final String DEFAULT_DATE_FORMAT = "dd-MMM-yyyy";
-  static final String DEFAULT_TIMESTAMP_FORMAT = "dd-MMM-yyyy HH:mm:ss";
-
-  /**
-   * Default Constructor.
-   */
-  public ResultSetHelperService() {
-  }
-
-  private static String read(Clob c) throws SQLException, IOException {
-    StringBuilder sb = new StringBuilder((int) c.length());
-    Reader r = c.getCharacterStream();
-    try {
-      char[] cbuf = new char[CLOBBUFFERSIZE];
-      int n;
-      while ((n = r.read(cbuf, 0, cbuf.length)) != -1) {
-        sb.append(cbuf, 0, n);
-      }
-    } finally {
-      r.close();
-    }
-    return sb.toString();
-
-  }
-
-  /**
-   * Returns the column names from the result set.
-   *
-   * @param rs - ResultSet
-   * @return - a string array containing the column names.
-   * @throws SQLException - thrown by the result set.
-   */
-  public String[] getColumnNames(ResultSet rs) throws SQLException {
-    List<String> names = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    //CHECKSTYLE:OFF    Approval No:Approval-V1R2C10_010
-    ResultSetMetaData metadata = rs.getMetaData();
-    //CHECKSTYLE:ON
-    for (int i = 0; i < metadata.getColumnCount(); i++) {
-      names.add(metadata.getColumnName(i + 1));
-    }
-
-    String[] nameArray = new String[names.size()];
-    return names.toArray(nameArray);
-  }
-
-  /**
-   * Get all the column values from the result set.
-   *
-   * @param rs - the ResultSet containing the values.
-   * @return - String array containing all the column values.
-   * @throws SQLException - thrown by the result set.
-   * @throws IOException  - thrown by the result set.
-   */
-  public String[] getColumnValues(ResultSet rs) throws SQLException, IOException {
-    return this.getColumnValues(rs, false, DEFAULT_DATE_FORMAT, DEFAULT_TIMESTAMP_FORMAT);
-  }
-
-  /**
-   * Get all the column values from the result set.
-   *
-   * @param rs   - the ResultSet containing the values.
-   * @param trim - values should have white spaces trimmed.
-   * @return - String array containing all the column values.
-   * @throws SQLException - thrown by the result set.
-   * @throws IOException  - thrown by the result set.
-   */
-  public String[] getColumnValues(ResultSet rs, boolean trim) throws SQLException, IOException {
-    return this.getColumnValues(rs, trim, DEFAULT_DATE_FORMAT, DEFAULT_TIMESTAMP_FORMAT);
-  }
-
-  /**
-   * Get all the column values from the result set.
-   *
-   * @param rs               - the ResultSet containing the values.
-   * @param trim             - values should have white spaces trimmed.
-   * @param dateFormatString - format String for dates.
-   * @param timeFormatString - format String for timestamps.
-   * @return - String array containing all the column values.
-   * @throws SQLException - thrown by the result set.
-   * @throws IOException  - thrown by the result set.
-   */
-  public String[] getColumnValues(ResultSet rs, boolean trim, String dateFormatString,
-      String timeFormatString) throws SQLException, IOException {
-    List<String> values = new ArrayList<>();
-    ResultSetMetaData metadata = rs.getMetaData();
-
-    for (int i = 0; i < metadata.getColumnCount(); i++) {
-      values.add(getColumnValue(rs, metadata.getColumnType(i + 1), i + 1, trim, dateFormatString,
-          timeFormatString));
-    }
-
-    String[] valueArray = new String[values.size()];
-    return values.toArray(valueArray);
-  }
-
-  /**
-   * changes an object to a String.
-   *
-   * @param obj - Object to format.
-   * @return - String value of an object or empty string if the object is null.
-   */
-  protected String handleObject(Object obj) {
-    return obj == null ? "" : String.valueOf(obj);
-  }
-
-  /**
-   * changes a BigDecimal to String.
-   *
-   * @param decimal - BigDecimal to format
-   * @return String representation of a BigDecimal or empty string if null
-   */
-  protected String handleBigDecimal(BigDecimal decimal) {
-    return decimal == null ? "" : decimal.toString();
-  }
-
-  /**
-   * Retrieves the string representation of an Long value from the result set.
-   *
-   * @param rs          - Result set containing the data.
-   * @param columnIndex - index to the column of the long.
-   * @return - the string representation of the long
-   * @throws SQLException - thrown by the result set on error.
-   */
-  protected String handleLong(ResultSet rs, int columnIndex) throws SQLException {
-    long lv = rs.getLong(columnIndex);
-    return rs.wasNull() ? "" : Long.toString(lv);
-  }
-
-  /**
-   * Retrieves the string representation of an Integer value from the result set.
-   *
-   * @param rs          - Result set containing the data.
-   * @param columnIndex - index to the column of the integer.
-   * @return - string representation of the Integer.
-   * @throws SQLException - returned from the result set on error.
-   */
-  protected String handleInteger(ResultSet rs, int columnIndex) throws SQLException {
-    int i = rs.getInt(columnIndex);
-    return rs.wasNull() ? "" : Integer.toString(i);
-  }
-
-  /**
-   * Retrieves a date from the result set.
-   *
-   * @param rs               - Result set containing the data
-   * @param columnIndex      - index to the column of the date
-   * @param dateFormatString - format for the date
-   * @return - formatted date.
-   * @throws SQLException - returned from the result set on error.
-   */
-  protected String handleDate(ResultSet rs, int columnIndex, String dateFormatString)
-      throws SQLException {
-    java.sql.Date date = rs.getDate(columnIndex);
-    String value = null;
-    if (date != null) {
-      SimpleDateFormat dateFormat = new SimpleDateFormat(dateFormatString);
-      value = dateFormat.format(date);
-    }
-    return value;
-  }
-
-  /**
-   * Return time read from ResultSet.
-   *
-   * @param time time read from ResultSet
-   * @return String version of time or null if time is null.
-   */
-  protected String handleTime(Time time) {
-    return time == null ? null : time.toString();
-  }
-
-  /**
-   * The formatted timestamp.
-   *
-   * @param timestamp             - timestamp read from resultset
-   * @param timestampFormatString - format string
-   * @return - formatted time stamp.
-   */
-  protected String handleTimestamp(Timestamp timestamp, String timestampFormatString) {
-    SimpleDateFormat timeFormat = new SimpleDateFormat(timestampFormatString);
-    return timestamp == null ? null : timeFormat.format(timestamp);
-  }
-
-  private String getColumnValue(ResultSet rs, int colType, int colIndex, boolean trim,
-      String dateFormatString, String timestampFormatString) throws SQLException, IOException {
-
-    String value = "";
-
-    switch (colType) {
-      case Types.BIT:
-      case Types.JAVA_OBJECT:
-        value = handleObject(rs.getObject(colIndex));
-        break;
-      case Types.BOOLEAN:
-        boolean b = rs.getBoolean(colIndex);
-        value = Boolean.valueOf(b).toString();
-        break;
-      case NCLOB: // todo : use rs.getNClob
-      case Types.CLOB:
-        Clob c = rs.getClob(colIndex);
-        if (c != null) {
-          value = read(c);
-        }
-        break;
-      case Types.BIGINT:
-        value = handleLong(rs, colIndex);
-        break;
-      case Types.DECIMAL:
-      case Types.DOUBLE:
-      case Types.FLOAT:
-      case Types.REAL:
-      case Types.NUMERIC:
-        value = handleBigDecimal(rs.getBigDecimal(colIndex));
-        break;
-      case Types.INTEGER:
-      case Types.TINYINT:
-      case Types.SMALLINT:
-        value = handleInteger(rs, colIndex);
-        break;
-      case Types.DATE:
-        value = handleDate(rs, colIndex, dateFormatString);
-        break;
-      case Types.TIME:
-        value = handleTime(rs.getTime(colIndex));
-        break;
-      case Types.TIMESTAMP:
-        value = handleTimestamp(rs.getTimestamp(colIndex), timestampFormatString);
-        break;
-      case NVARCHAR: // todo : use rs.getNString
-      case NCHAR: // todo : use rs.getNString
-      case LONGNVARCHAR: // todo : use rs.getNString
-      case Types.LONGVARCHAR:
-      case Types.VARCHAR:
-      case Types.CHAR:
-        value = getColumnValue(rs, colIndex, trim);
-        break;
-      default:
-        value = "";
-    }
-
-    if (value == null) {
-      value = "";
-    }
-
-    return value;
-  }
-
-  /**
-   * @param rs
-   * @param colIndex
-   * @param trim
-   * @return
-   * @throws SQLException
-   */
-  public String getColumnValue(ResultSet rs, int colIndex, boolean trim) throws SQLException {
-    String value;
-    String columnValue = rs.getString(colIndex);
-    if (trim && columnValue != null) {
-      value = columnValue.trim();
-    } else {
-      value = columnValue;
-    }
-    return value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataPartitionRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataPartitionRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataPartitionRDD.scala
deleted file mode 100644
index d6932cd..0000000
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataPartitionRDD.scala
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.carbondata.spark.rdd
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.{Logging, Partition, SerializableWritable, SparkContext, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.execution.command.Partitioner
-
-import org.apache.carbondata.common.logging.impl.StandardLogService
-import org.apache.carbondata.spark.PartitionResult
-import org.apache.carbondata.spark.partition.api.impl.CSVFilePartitioner
-import org.apache.carbondata.spark.splits.TableSplit
-import org.apache.carbondata.spark.util.CarbonQueryUtil
-
-
-class CarbonSparkRawDataPartition(rddId: Int, val idx: Int, @transient val tableSplit: TableSplit)
-  extends Partition {
-
-  override val index: Int = idx
-  val serializableHadoopSplit = new SerializableWritable[TableSplit](tableSplit)
-
-  override def hashCode(): Int = 41 * (41 + rddId) + idx
-}
-
-/**
- * This RDD class is used to  create splits the fact csv store to various partitions as per
- * configuration  and compute each split in the respective node located in the server.
- * .
- */
-class CarbonDataPartitionRDD[K, V](
-    sc: SparkContext,
-    results: PartitionResult[K, V],
-    databaseName: String,
-    tableName: String,
-    sourcePath: String,
-    targetFolder: String,
-    requiredColumns: Array[String],
-    headers: String,
-    delimiter: String,
-    quoteChar: String,
-    escapeChar: String,
-    multiLine: Boolean,
-    partitioner: Partitioner)
-  extends RDD[(K, V)](sc, Nil) with Logging {
-
-  sc.setLocalProperty("spark.scheduler.pool", "DDL")
-
-  override def getPartitions: Array[Partition] = {
-    val splits = CarbonQueryUtil
-      .getPartitionSplits(sourcePath, partitioner.nodeList, partitioner.partitionCount)
-    splits.zipWithIndex.map {s =>
-      new CarbonSparkRawDataPartition(id, s._2, s._1)
-    }
-  }
-
-  override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
-    new Iterator[(K, V)] {
-      val split = theSplit.asInstanceOf[CarbonSparkRawDataPartition]
-      StandardLogService
-        .setThreadName(split.serializableHadoopSplit.value.getPartition.getUniqueID, null)
-      logInfo("Input split: " + split.serializableHadoopSplit.value)
-
-      val csvPart = new CSVFilePartitioner(partitioner.partitionClass, sourcePath)
-      csvPart.splitFile(databaseName, tableName,
-        split.serializableHadoopSplit.value.getPartition.getFilesPath, targetFolder,
-        partitioner.nodeList.toList.asJava, partitioner.partitionCount, partitioner.partitionColumn,
-        requiredColumns, delimiter, quoteChar, headers, escapeChar, multiLine)
-
-      var finished = false
-
-      override def hasNext: Boolean = {
-        if (!finished) {
-          finished = true
-          finished
-        }
-        else {
-          !finished
-        }
-      }
-
-      override def next(): (K, V) = {
-        results.getKey(partitioner.partitionCount, csvPart.isPartialSuccess)
-      }
-    }
-  }
-
-  override def getPreferredLocations(split: Partition): Seq[String] = {
-    val theSplit = split.asInstanceOf[CarbonSparkRawDataPartition]
-    val s = theSplit.serializableHadoopSplit.value.getLocations.asScala
-    logInfo("Host Name : " + s.head + s.length)
-    s
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/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 3118d3f..964c955 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
@@ -62,37 +62,6 @@ object CarbonDataRDDFactory extends Logging {
 
   val logger = LogServiceFactory.getLogService(CarbonDataRDDFactory.getClass.getName)
 
-  // scalastyle:off
-  def partitionCarbonData(sc: SparkContext,
-      databaseName: String,
-      tableName: String,
-      sourcePath: String,
-      targetFolder: String,
-      requiredColumns: Array[String],
-      headers: String,
-      delimiter: String,
-      quoteChar: String,
-      escapeChar: String,
-      multiLine: Boolean,
-      partitioner: Partitioner): String = {
-    // scalastyle:on
-    val status = new
-        CarbonDataPartitionRDD(sc, new PartitionResultImpl(), databaseName, tableName, sourcePath,
-          targetFolder, requiredColumns, headers, delimiter, quoteChar, escapeChar, multiLine,
-          partitioner
-        ).collect
-    CarbonDataProcessorUtil
-      .renameBadRecordsFromInProgressToNormal("partition/" + databaseName + '/' + tableName)
-    var loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
-    status.foreach {
-      case (key, value) =>
-        if (value) {
-          loadStatus = CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS
-        }
-    }
-    loadStatus
-  }
-
   def mergeCarbonData(
       sqlContext: SQLContext,
       carbonLoadModel: CarbonLoadModel,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
index 59db05b..a7d807e 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
@@ -25,6 +25,7 @@ import scala.collection.mutable
 import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet}
 import scala.util.control.Breaks.{break, breakable}
 
+import au.com.bytecode.opencsv.CSVReader
 import org.apache.commons.lang3.{ArrayUtils, StringUtils}
 import org.apache.spark._
 import org.apache.spark.rdd.RDD
@@ -38,9 +39,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory}
 import org.apache.carbondata.lcm.locks.{CarbonLockFactory, LockUsage}
-import org.apache.carbondata.spark.load.CarbonLoaderUtil
-import org.apache.carbondata.spark.load.CarbonLoadModel
-import org.apache.carbondata.spark.partition.reader.{CSVParser, CSVReader}
+import org.apache.carbondata.spark.load.{CarbonLoaderUtil, CarbonLoadModel}
 import org.apache.carbondata.spark.tasks.{DictionaryWriterTask, SortIndexWriterTask}
 import org.apache.carbondata.spark.util.GlobalDictionaryUtil
 import org.apache.carbondata.spark.util.GlobalDictionaryUtil._
@@ -500,7 +499,7 @@ class CarbonColumnDictGenerateRDD(carbonLoadModel: CarbonLoadModel,
       inputStream = FileFactory.getDataInputStream(preDefDictFilePath,
         FileFactory.getFileType(preDefDictFilePath))
       csvReader = new CSVReader(new InputStreamReader(inputStream, Charset.defaultCharset),
-        CSVReader.DEFAULT_SKIP_LINES, new CSVParser(carbonLoadModel.getCsvDelimiter.charAt(0)))
+        carbonLoadModel.getCsvDelimiter.charAt(0))
       // read the column data to list iterator
       colDictData = csvReader.readAll.iterator
     } catch {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index 818aa4a..e714520 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -49,7 +49,6 @@ import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.carbondata.spark.CarbonSparkFactory
 import org.apache.carbondata.spark.load.CarbonLoaderUtil
 import org.apache.carbondata.spark.load.CarbonLoadModel
-import org.apache.carbondata.spark.partition.reader.CSVWriter
 import org.apache.carbondata.spark.rdd._
 
 /**
@@ -58,6 +57,16 @@ import org.apache.carbondata.spark.rdd._
 object GlobalDictionaryUtil extends Logging {
 
   /**
+   * The default separator to use if none is supplied to the constructor.
+   */
+  val DEFAULT_SEPARATOR: Char = ','
+  /**
+   * The default quote character to use if none is supplied to the
+   * constructor.
+   */
+  val DEFAULT_QUOTE_CHARACTER: Char = '"'
+
+  /**
    * find columns which need to generate global dictionary.
    *
    * @param dimensions  dimension list of schema
@@ -354,7 +363,7 @@ object GlobalDictionaryUtil extends Logging {
       })
       .option("delimiter", {
         if (StringUtils.isEmpty(carbonLoadModel.getCsvDelimiter)) {
-          "" + CSVWriter.DEFAULT_SEPARATOR
+          "" + DEFAULT_SEPARATOR
         }
         else {
           carbonLoadModel.getCsvDelimiter
@@ -367,7 +376,7 @@ object GlobalDictionaryUtil extends Logging {
       .option("codec", "gzip")
       .option("quote", {
         if (StringUtils.isEmpty(carbonLoadModel.getQuoteChar)) {
-          "" + CSVWriter. DEFAULT_QUOTE_CHARACTER
+          "" + DEFAULT_QUOTE_CHARACTER
         }
         else {
           carbonLoadModel.getQuoteChar
@@ -592,7 +601,7 @@ object GlobalDictionaryUtil extends Logging {
    */
   private def parseRecord(x: String, accum: Accumulator[Int],
                   csvFileColumns: Array[String]) : (String, String) = {
-    val tokens = x.split("" + CSVWriter.DEFAULT_SEPARATOR)
+    val tokens = x.split("" + DEFAULT_SEPARATOR)
     var columnName: String = ""
     var value: String = ""
     // such as "," , "", throw ex
@@ -713,7 +722,7 @@ object GlobalDictionaryUtil extends Logging {
 
     if (null != readLine) {
       val delimiter = if (StringUtils.isEmpty(carbonLoadModel.getCsvDelimiter)) {
-        "" + CSVWriter.DEFAULT_SEPARATOR
+        "" + DEFAULT_SEPARATOR
       } else {
         carbonLoadModel.getCsvDelimiter
       }
@@ -756,7 +765,7 @@ object GlobalDictionaryUtil extends Logging {
           df.columns
         }
         else {
-          carbonLoadModel.getCsvHeader.split("" + CSVWriter.DEFAULT_SEPARATOR)
+          carbonLoadModel.getCsvHeader.split("" + DEFAULT_SEPARATOR)
         }
         headers = headers.map(headerName => headerName.trim)
         val colDictFilePath = carbonLoadModel.getColDictFilePath
@@ -820,7 +829,7 @@ object GlobalDictionaryUtil extends Logging {
           var headers = if (StringUtils.isEmpty(carbonLoadModel.getCsvHeader)) {
             getHeaderFormFactFile(carbonLoadModel)
           } else {
-            carbonLoadModel.getCsvHeader.toLowerCase.split("" + CSVWriter.DEFAULT_SEPARATOR)
+            carbonLoadModel.getCsvHeader.toLowerCase.split("" + DEFAULT_SEPARATOR)
           }
           headers = headers.map(headerName => headerName.trim)
           // prune columns according to the CSV file header, dimension columns

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
index 972de05..c9fdd6b 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
@@ -28,13 +28,12 @@ import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog}
 import org.apache.spark.sql.catalyst.optimizer.Optimizer
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.ExtractPythonUDFs
-import org.apache.spark.sql.execution.command.PartitionData
 import org.apache.spark.sql.execution.datasources.{PreInsertCastAndRename, PreWriteCheck}
 import org.apache.spark.sql.hive._
 import org.apache.spark.sql.optimizer.CarbonOptimizer
 
 import org.apache.carbondata.common.logging.LogServiceFactory
-import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
+import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory}
 import org.apache.carbondata.spark.rdd.CarbonDataFrameRDD
@@ -153,45 +152,6 @@ object CarbonContext {
 
   @transient
   val LOGGER = LogServiceFactory.getLogService(CarbonContext.getClass.getName)
-  /**
-   * @param databaseName - Database Name
-   * @param tableName   - Table Name
-   * @param factPath   - Raw CSV data path
-   * @param targetPath - Target path where the file will be split as per partition
-   * @param delimiter  - default file delimiter is comma(,)
-   * @param quoteChar  - default quote character used in Raw CSV file, Default quote
-   *                   character is double quote(")
-   * @param fileHeader - Header should be passed if not available in Raw CSV File, else pass null,
-   *                   Header will be read from CSV
-   * @param escapeChar - This parameter by default will be null, there wont be any validation if
-   *                   default escape character(\) is found on the RawCSV file
-   * @param multiLine  - This parameter will be check for end of quote character if escape character
-   *                   & quote character is set.
-   *                   if set as false, it will check for end of quote character within the line
-   *                   and skips only 1 line if end of quote not found
-   *                   if set as true, By default it will check for 10000 characters in multiple
-   *                   lines for end of quote & skip all lines if end of quote not found.
-   */
-  final def partitionData(
-      databaseName: String = null,
-      tableName: String,
-      factPath: String,
-      targetPath: String,
-      delimiter: String = ",",
-      quoteChar: String = "\"",
-      fileHeader: String = null,
-      escapeChar: String = null,
-      multiLine: Boolean = false)(hiveContext: HiveContext): String = {
-    updateCarbonPorpertiesPath(hiveContext)
-    var databaseNameLocal = databaseName
-    if (databaseNameLocal == null) {
-      databaseNameLocal = "default"
-    }
-    val partitionDataClass = PartitionData(databaseName, tableName, factPath, targetPath, delimiter,
-      quoteChar, fileHeader, escapeChar, multiLine)
-    partitionDataClass.run(hiveContext)
-    partitionDataClass.partitionStatus
-  }
 
   final def updateCarbonPorpertiesPath(hiveContext: HiveContext) {
     val carbonPropertiesFilePath = hiveContext.getConf("carbon.properties.filepath", null)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/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 e4a79ab..ac48624 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
@@ -1139,28 +1139,6 @@ private[sql] case class LoadTable(
           carbonLoadModel.setColDictFilePath(columnDict)
           carbonLoadModel.setDirectLoad(true)
         }
-        else {
-          val fileType = FileFactory.getFileType(partitionLocation)
-          if (FileFactory.isFileExist(partitionLocation, fileType)) {
-            val file = FileFactory.getCarbonFile(partitionLocation, fileType)
-            CarbonUtil.deleteFoldersAndFiles(file)
-          }
-          partitionLocation += System.currentTimeMillis()
-          FileFactory.mkdirs(partitionLocation, fileType)
-          LOGGER.info("Initiating Data Partitioning for the Table : (" +
-                      dbName + "." + tableName + ")")
-          partitionStatus = CarbonContext.partitionData(
-            dbName,
-            tableName,
-            factPath,
-            partitionLocation,
-            delimiter,
-            quoteChar,
-            fileHeader,
-            escapeChar, multiLine)(sqlContext.asInstanceOf[HiveContext])
-          carbonLoadModel.setFactFilePath(FileUtils.getPaths(partitionLocation))
-          carbonLoadModel.setColDictFilePath(columnDict)
-        }
         GlobalDictionaryUtil
           .generateGlobalDictionary(sqlContext, carbonLoadModel, relation.tableMeta.storePath)
         CarbonDataRDDFactory
@@ -1213,34 +1191,6 @@ private[sql] case class LoadTable(
 
 }
 
-private[sql] case class PartitionData(databaseName: String, tableName: String, factPath: String,
-    targetPath: String, delimiter: String, quoteChar: String,
-    fileHeader: String, escapeChar: String, multiLine: Boolean)
-  extends RunnableCommand {
-
-  var partitionStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
-
-  def run(sqlContext: SQLContext): Seq[Row] = {
-    val identifier = TableIdentifier(tableName, Option(databaseName))
-    val relation = CarbonEnv.getInstance(sqlContext)
-      .carbonCatalog.lookupRelation1(identifier, None)(sqlContext).asInstanceOf[CarbonRelation]
-    val dimNames = relation.tableMeta.carbonTable
-      .getDimensionByTableName(tableName).asScala.map(_.getColName)
-    val msrNames = relation.tableMeta.carbonTable
-      .getDimensionByTableName(tableName).asScala.map(_.getColName)
-    val targetFolder = targetPath
-    partitionStatus = CarbonDataRDDFactory.partitionCarbonData(
-      sqlContext.sparkContext, databaseName,
-      tableName, factPath, targetFolder, (dimNames ++ msrNames).toArray
-      , fileHeader, delimiter,
-      quoteChar, escapeChar, multiLine, relation.tableMeta.partitioner)
-    if (partitionStatus == CarbonCommonConstants.STORE_LOADSTATUS_PARTIAL_SUCCESS) {
-      logInfo("Bad Record Found while partitioning data")
-    }
-    Seq.empty
-  }
-}
-
 private[sql] case class DropTableCommand(ifExistsSet: Boolean, databaseNameOp: Option[String],
     tableName: String)
   extends RunnableCommand {


[48/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-214]Remove binary file and generate the binary file on fly while running the test case This closes 179

Posted by ra...@apache.org.
[CARBONDATA-214]Remove binary file and generate the binary file on fly while running the test case This closes 179


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

Branch: refs/heads/branch-0.1
Commit: b55e7d36d48b78ca43f8e70acc51361c38be46f5
Parents: 263ac7a
Author: chenliang613 <ch...@apache.org>
Authored: Wed Sep 21 09:30:10 2016 -0700
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:37:55 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[46/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-215][CARBONDATA-216] Corrected file headers and removed unnecessary open csv files This closes #186

Posted by ra...@apache.org.
[CARBONDATA-215][CARBONDATA-216] Corrected file headers and removed unnecessary open csv files This closes #186


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

Branch: refs/heads/branch-0.1
Commit: a01917ba17db6adf4017b7fa37df815ab1680ad7
Parents: be1675f
Author: chenliang613 <ch...@apache.org>
Authored: Wed Sep 21 09:24:56 2016 -0700
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:35:37 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[31/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-250] Double Filter = -0.0 not matching 0.0. This closes #169

Posted by ra...@apache.org.
[CARBONDATA-250] Double Filter = -0.0 not matching 0.0. This closes #169


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

Branch: refs/heads/branch-0.1
Commit: 9b415d7f430cb786386a752bd2784f435fc4c420
Parents: c083264
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sun Sep 18 04:44:37 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:02:29 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[25/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-200] change statistics to milliseconds. This closes #138

Posted by ra...@apache.org.
[CARBONDATA-200] change statistics to milliseconds. This closes #138


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

Branch: refs/heads/branch-0.1
Commit: 14029e2fb83a96b9e503cc5461cd268ca9bb48a1
Parents: 956d988
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Thu Sep 8 13:21:50 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:58:41 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[28/50] [abbrv] incubator-carbondata git commit: equalsAndHashCodeIssue

Posted by ra...@apache.org.
equalsAndHashCodeIssue


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

Branch: refs/heads/branch-0.1
Commit: 303ccd93371b4234332cde17c38f71def369661b
Parents: cc5ca87
Author: kumarvishal <ku...@gmail.com>
Authored: Sat Sep 17 18:55:36 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:01:25 2016 +0530

----------------------------------------------------------------------
 .../core/carbon/datastore/BlockIndexStore.java  |  73 +++++++------
 .../core/carbon/datastore/block/BlockInfo.java  | 107 +++++++++++++++++++
 .../carbon/datastore/block/TableBlockInfo.java  |  18 ++--
 .../impl/btree/BlockBTreeLeafNode.java          |   7 +-
 .../impl/btree/BlockletBTreeLeafNode.java       |   4 +-
 .../metadata/blocklet/DataFileFooter.java       |  12 +--
 .../core/util/DataFileFooterConverter.java      |   3 +-
 7 files changed, 169 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/303ccd93/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
index 4a36373..bbebda0 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
@@ -31,11 +31,10 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.apache.carbondata.core.carbon.datastore.block.AbstractIndex;
 import org.apache.carbondata.core.carbon.datastore.block.BlockIndex;
+import org.apache.carbondata.core.carbon.datastore.block.BlockInfo;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.datastore.exception.IndexBuilderException;
 import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
@@ -50,8 +49,6 @@ import org.apache.carbondata.core.util.CarbonUtilException;
  */
 public class BlockIndexStore {
 
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(BlockIndexStore.class.getName());
   /**
    * singleton instance
    */
@@ -60,13 +57,13 @@ public class BlockIndexStore {
   /**
    * map to hold the table and its list of blocks
    */
-  private Map<AbsoluteTableIdentifier, Map<TableBlockInfo, AbstractIndex>> tableBlocksMap;
+  private Map<AbsoluteTableIdentifier, Map<BlockInfo, AbstractIndex>> tableBlocksMap;
 
   /**
    * map to maintain segment id to block info map, this map will be used to
    * while removing the block from memory when segment is compacted or deleted
    */
-  private Map<AbsoluteTableIdentifier, Map<String, List<TableBlockInfo>>> segmentIdToBlockListMap;
+  private Map<AbsoluteTableIdentifier, Map<String, List<BlockInfo>>> segmentIdToBlockListMap;
 
   /**
    * map of block info to lock object map, while loading the btree this will be filled
@@ -74,7 +71,7 @@ public class BlockIndexStore {
    * while loading the tree concurrently so only block level lock will be applied another
    * block can be loaded concurrently
    */
-  private Map<TableBlockInfo, Object> blockInfoLock;
+  private Map<BlockInfo, Object> blockInfoLock;
 
   /**
    * table and its lock object to this will be useful in case of concurrent
@@ -84,12 +81,11 @@ public class BlockIndexStore {
   private Map<AbsoluteTableIdentifier, Object> tableLockMap;
 
   private BlockIndexStore() {
-    tableBlocksMap =
-        new ConcurrentHashMap<AbsoluteTableIdentifier, Map<TableBlockInfo, AbstractIndex>>(
-            CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    tableBlocksMap = new ConcurrentHashMap<AbsoluteTableIdentifier, Map<BlockInfo, AbstractIndex>>(
+        CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     tableLockMap = new ConcurrentHashMap<AbsoluteTableIdentifier, Object>(
         CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    blockInfoLock = new ConcurrentHashMap<TableBlockInfo, Object>();
+    blockInfoLock = new ConcurrentHashMap<BlockInfo, Object>();
     segmentIdToBlockListMap = new ConcurrentHashMap<>();
   }
 
@@ -118,7 +114,7 @@ public class BlockIndexStore {
 
     // get the instance
     Object lockObject = tableLockMap.get(absoluteTableIdentifier);
-    Map<TableBlockInfo, AbstractIndex> tableBlockMapTemp = null;
+    Map<BlockInfo, AbstractIndex> tableBlockMapTemp = null;
     int numberOfCores = 1;
     try {
       numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
@@ -130,19 +126,20 @@ public class BlockIndexStore {
     ExecutorService executor = Executors.newFixedThreadPool(numberOfCores);
     // Acquire the lock to ensure only one query is loading the table blocks
     // if same block is assigned to both the queries
+    List<BlockInfo> blockInfosNeedToLoad = null;
     synchronized (lockObject) {
       tableBlockMapTemp = tableBlocksMap.get(absoluteTableIdentifier);
       // if it is loading for first time
       if (null == tableBlockMapTemp) {
-        tableBlockMapTemp = new ConcurrentHashMap<TableBlockInfo, AbstractIndex>();
+        tableBlockMapTemp = new ConcurrentHashMap<BlockInfo, AbstractIndex>();
         tableBlocksMap.put(absoluteTableIdentifier, tableBlockMapTemp);
       }
-      fillSegmentIdToTableInfoMap(tableBlocksInfos, absoluteTableIdentifier);
+      blockInfosNeedToLoad = fillSegmentIdToTableInfoMap(tableBlocksInfos, absoluteTableIdentifier);
     }
     AbstractIndex tableBlock = null;
     List<Future<AbstractIndex>> blocksList = new ArrayList<Future<AbstractIndex>>();
     int counter = -1;
-    for (TableBlockInfo blockInfo : tableBlocksInfos) {
+    for (BlockInfo blockInfo : blockInfosNeedToLoad) {
       counter++;
       // if table block is already loaded then do not load
       // that block
@@ -204,23 +201,29 @@ public class BlockIndexStore {
    * @param tableBlockInfos         table block infos
    * @param absoluteTableIdentifier absolute table identifier
    */
-  private void fillSegmentIdToTableInfoMap(List<TableBlockInfo> tableBlockInfos,
+  private List<BlockInfo> fillSegmentIdToTableInfoMap(List<TableBlockInfo> tableBlockInfos,
       AbsoluteTableIdentifier absoluteTableIdentifier) {
-    Map<String, List<TableBlockInfo>> map = segmentIdToBlockListMap.get(absoluteTableIdentifier);
+    Map<String, List<BlockInfo>> map = segmentIdToBlockListMap.get(absoluteTableIdentifier);
     if (null == map) {
-      map = new ConcurrentHashMap<String, List<TableBlockInfo>>();
+      map = new ConcurrentHashMap<String, List<BlockInfo>>();
       segmentIdToBlockListMap.put(absoluteTableIdentifier, map);
     }
+    BlockInfo temp = null;
+    List<BlockInfo> blockInfosNeedToLoad = new ArrayList<>();
+
     for (TableBlockInfo info : tableBlockInfos) {
-      List<TableBlockInfo> tempTableBlockInfos = map.get(info.getSegmentId());
+      List<BlockInfo> tempTableBlockInfos = map.get(info.getSegmentId());
       if (null == tempTableBlockInfos) {
         tempTableBlockInfos = new ArrayList<>();
         map.put(info.getSegmentId(), tempTableBlockInfos);
       }
-      if (!tempTableBlockInfos.contains(info)) {
-        tempTableBlockInfos.add(info);
+      temp = new BlockInfo(info);
+      if (!tempTableBlockInfos.contains(temp)) {
+        tempTableBlockInfos.add(temp);
       }
+      blockInfosNeedToLoad.add(temp);
     }
+    return blockInfosNeedToLoad;
   }
 
   /**
@@ -246,15 +249,16 @@ public class BlockIndexStore {
     }
   }
 
-  private AbstractIndex loadBlock(Map<TableBlockInfo, AbstractIndex> tableBlockMapTemp,
-      TableBlockInfo blockInfo) throws CarbonUtilException {
+  private AbstractIndex loadBlock(Map<BlockInfo, AbstractIndex> tableBlockMapTemp,
+      BlockInfo blockInfo) throws CarbonUtilException {
     AbstractIndex tableBlock;
     DataFileFooter footer;
     // getting the data file meta data of the block
-    footer = CarbonUtil.readMetadatFile(blockInfo.getFilePath(), blockInfo.getBlockOffset(),
-        blockInfo.getBlockLength());
+    footer = CarbonUtil.readMetadatFile(blockInfo.getTableBlockInfo().getFilePath(),
+        blockInfo.getTableBlockInfo().getBlockOffset(),
+        blockInfo.getTableBlockInfo().getBlockLength());
     tableBlock = new BlockIndex();
-    footer.setTableBlockInfo(blockInfo);
+    footer.setBlockInfo(blockInfo);
     // building the block
     tableBlock.buildIndex(Arrays.asList(footer));
     tableBlockMapTemp.put(blockInfo, tableBlock);
@@ -293,25 +297,25 @@ public class BlockIndexStore {
     if (null == lockObject) {
       return;
     }
-    Map<TableBlockInfo, AbstractIndex> map = tableBlocksMap.get(absoluteTableIdentifier);
+    Map<BlockInfo, AbstractIndex> map = tableBlocksMap.get(absoluteTableIdentifier);
     // if there is no loaded blocks then return
     if (null == map || map.isEmpty()) {
       return;
     }
-    Map<String, List<TableBlockInfo>> segmentIdToBlockInfoMap =
+    Map<String, List<BlockInfo>> segmentIdToBlockInfoMap =
         segmentIdToBlockListMap.get(absoluteTableIdentifier);
     if (null == segmentIdToBlockInfoMap || segmentIdToBlockInfoMap.isEmpty()) {
       return;
     }
     synchronized (lockObject) {
       for (String segmentId : segmentsToBeRemoved) {
-        List<TableBlockInfo> tableBlockInfoList = segmentIdToBlockInfoMap.remove(segmentId);
+        List<BlockInfo> tableBlockInfoList = segmentIdToBlockInfoMap.remove(segmentId);
         if (null == tableBlockInfoList) {
           continue;
         }
-        Iterator<TableBlockInfo> tableBlockInfoIterator = tableBlockInfoList.iterator();
+        Iterator<BlockInfo> tableBlockInfoIterator = tableBlockInfoList.iterator();
         while (tableBlockInfoIterator.hasNext()) {
-          TableBlockInfo info = tableBlockInfoIterator.next();
+          BlockInfo info = tableBlockInfoIterator.next();
           map.remove(info);
         }
       }
@@ -336,13 +340,12 @@ public class BlockIndexStore {
     /**
      * table block info to block index map
      */
-    private Map<TableBlockInfo, AbstractIndex> tableBlockMap;
+    private Map<BlockInfo, AbstractIndex> tableBlockMap;
 
     // block info
-    private TableBlockInfo blockInfo;
+    private BlockInfo blockInfo;
 
-    private BlockLoaderThread(TableBlockInfo blockInfo,
-        Map<TableBlockInfo, AbstractIndex> tableBlockMap) {
+    private BlockLoaderThread(BlockInfo blockInfo, Map<BlockInfo, AbstractIndex> tableBlockMap) {
       this.tableBlockMap = tableBlockMap;
       this.blockInfo = blockInfo;
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/303ccd93/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfo.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfo.java
new file mode 100644
index 0000000..8092c7c
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/BlockInfo.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.datastore.block;
+
+/**
+ * Below class will be used to store table block info
+ * As in blocklet distribution we are dividing the same block
+ * in parts but in case of block loading blocklets belongs to same
+ * block will be loaded together. This class will be used to store table block info
+ * and equals and hash code method is used to identify blocklet belongs to same block
+ */
+public class BlockInfo {
+
+  /**
+   * table block info, stores all the details
+   * about the block
+   */
+  private TableBlockInfo info;
+
+  /**
+   * Constructor
+   *
+   * @param info
+   */
+  public BlockInfo(TableBlockInfo info) {
+    this.info = info;
+  }
+
+  /**
+   * @return table Block info
+   */
+  public TableBlockInfo getTableBlockInfo() {
+    return info;
+  }
+
+  /**
+   * To set the table block info
+   *
+   * @param info
+   */
+  public void setTableBlockInfo(TableBlockInfo info) {
+    this.info = info;
+  }
+
+  /**
+   * method to get the hash code
+   */
+  @Override public int hashCode() {
+    int result = info.getFilePath().hashCode();
+    result = 31 * result + (int) (info.getBlockOffset() ^ (info.getBlockOffset() >>> 32));
+    result = 31 * result + (int) (info.getBlockLength() ^ (info.getBlockLength() >>> 32));
+    result = 31 * result + info.getSegmentId().hashCode();
+    return result;
+  }
+
+  /**
+   * To check the equality
+   *
+   * @param obj
+   */
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof BlockInfo)) {
+      return false;
+    }
+    BlockInfo other = (BlockInfo) obj;
+    if (!info.getSegmentId().equals(other.info.getSegmentId())) {
+      return false;
+    }
+    if (info.getBlockOffset() != other.info.getBlockOffset()) {
+      return false;
+    }
+    if (info.getBlockLength() != info.getBlockLength()) {
+      return false;
+    }
+
+    if (info.getFilePath() == null && other.info.getFilePath() != null) {
+      return false;
+    } else if (info.getFilePath() != null && other.info.getFilePath() == null) {
+      return false;
+    } else if (!info.getFilePath().equals(other.info.getFilePath())) {
+      return false;
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/303ccd93/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
index 3d393b6..f8da9af 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/block/TableBlockInfo.java
@@ -19,7 +19,6 @@
 package org.apache.carbondata.core.carbon.datastore.block;
 
 import java.io.Serializable;
-import java.util.Arrays;
 
 import org.apache.carbondata.core.carbon.path.CarbonTablePath;
 import org.apache.carbondata.core.carbon.path.CarbonTablePath.DataFileUtil;
@@ -74,6 +73,7 @@ public class TableBlockInfo extends Distributable
 
   /**
    * constructor to initialize the TbaleBlockInfo with BlockletInfos
+   *
    * @param filePath
    * @param blockOffset
    * @param segmentId
@@ -105,7 +105,6 @@ public class TableBlockInfo extends Distributable
     return blockOffset;
   }
 
-
   /**
    * @return the segmentId
    */
@@ -145,14 +144,16 @@ public class TableBlockInfo extends Distributable
     if (blockLength != other.blockLength) {
       return false;
     }
-
-    if (filePath == null) {
-      if (other.filePath != null) {
-        return false;
-      }
+    if (filePath == null && other.filePath != null) {
+      return false;
+    } else if (filePath != null && other.filePath == null) {
+      return false;
     } else if (!filePath.equals(other.filePath)) {
       return false;
     }
+    if (blockletInfos.getStartBlockletNumber() != other.blockletInfos.getStartBlockletNumber()) {
+      return false;
+    }
     return true;
   }
 
@@ -225,7 +226,6 @@ public class TableBlockInfo extends Distributable
     result = 31 * result + (int) (blockOffset ^ (blockOffset >>> 32));
     result = 31 * result + (int) (blockLength ^ (blockLength >>> 32));
     result = 31 * result + segmentId.hashCode();
-    result = 31 * result + Arrays.hashCode(locations);
     result = 31 * result + blockletInfos.getStartBlockletNumber();
     return result;
   }
@@ -236,6 +236,7 @@ public class TableBlockInfo extends Distributable
 
   /**
    * returns BlockletInfos
+   *
    * @return
    */
   public BlockletInfos getBlockletInfos() {
@@ -244,6 +245,7 @@ public class TableBlockInfo extends Distributable
 
   /**
    * set the blocklestinfos
+   *
    * @param blockletInfos
    */
   public void setBlockletInfos(BlockletInfos blockletInfos) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/303ccd93/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java
index fcc98c2..34468a6 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockBTreeLeafNode.java
@@ -19,6 +19,7 @@
 package org.apache.carbondata.core.carbon.datastore.impl.btree;
 
 import org.apache.carbondata.core.carbon.datastore.BTreeBuilderInfo;
+import org.apache.carbondata.core.carbon.datastore.block.BlockInfo;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
 import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxIndex;
@@ -30,7 +31,7 @@ import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxI
  */
 public class BlockBTreeLeafNode extends AbstractBTreeLeafNode {
 
-  private TableBlockInfo blockInfo;
+  private BlockInfo blockInfo;
 
   /**
    * Create a leaf node
@@ -47,7 +48,7 @@ public class BlockBTreeLeafNode extends AbstractBTreeLeafNode {
     minKeyOfColumns = minMaxIndex.getMinValues();
     numberOfKeys = 1;
     this.nodeNumber = nodeNumber;
-    this.blockInfo = footer.getTableBlockInfo();
+    this.blockInfo = footer.getBlockInfo();
   }
 
   /**
@@ -58,7 +59,7 @@ public class BlockBTreeLeafNode extends AbstractBTreeLeafNode {
    * @return TableBlockInfo
    */
   public TableBlockInfo getTableBlockInfo() {
-    return blockInfo;
+    return blockInfo.getTableBlockInfo();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/303ccd93/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
index 2bbddda..4293610 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/impl/btree/BlockletBTreeLeafNode.java
@@ -70,7 +70,7 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
     dimensionChunksReader = new CompressedDimensionChunkFileBasedReader(
         builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
             .getDimensionColumnChunk(), builderInfos.getDimensionColumnValueSize(),
-        builderInfos.getFooterList().get(0).getTableBlockInfo().getFilePath());
+        builderInfos.getFooterList().get(0).getBlockInfo().getTableBlockInfo().getFilePath());
     // get the value compression model which was used to compress the measure values
     ValueCompressionModel valueCompressionModel = CarbonUtil.getValueCompressionModel(
         builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
@@ -79,7 +79,7 @@ public class BlockletBTreeLeafNode extends AbstractBTreeLeafNode {
     measureColumnChunkReader = new CompressedMeasureChunkFileBasedReader(
         builderInfos.getFooterList().get(0).getBlockletList().get(leafIndex)
             .getMeasureColumnChunk(), valueCompressionModel,
-            builderInfos.getFooterList().get(0).getTableBlockInfo().getFilePath());
+            builderInfos.getFooterList().get(0).getBlockInfo().getTableBlockInfo().getFilePath());
     this.nodeNumber = nodeNumber;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/303ccd93/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
index 55587da..d4741eb 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/blocklet/DataFileFooter.java
@@ -21,7 +21,7 @@ package org.apache.carbondata.core.carbon.metadata.blocklet;
 import java.io.Serializable;
 import java.util.List;
 
-import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.carbon.datastore.block.BlockInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 
@@ -68,7 +68,7 @@ public class DataFileFooter implements Serializable {
   /**
    * to store the block info detail like file name block index and locations
    */
-  private TableBlockInfo tableBlockInfo;
+  private BlockInfo blockInfo;
 
   /**
    * @return the versionId
@@ -157,14 +157,14 @@ public class DataFileFooter implements Serializable {
   /**
    * @return the tableBlockInfo
    */
-  public TableBlockInfo getTableBlockInfo() {
-    return tableBlockInfo;
+  public BlockInfo getBlockInfo() {
+    return blockInfo;
   }
 
   /**
    * @param tableBlockInfo the tableBlockInfo to set
    */
-  public void setTableBlockInfo(TableBlockInfo tableBlockInfo) {
-    this.tableBlockInfo = tableBlockInfo;
+  public void setBlockInfo(BlockInfo tableBlockInfo) {
+    this.blockInfo = tableBlockInfo;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/303ccd93/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
index 8d7e893..5f3565c 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataFileFooterConverter.java
@@ -29,6 +29,7 @@ import java.util.List;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.carbon.datastore.block.BlockInfo;
 import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
 import org.apache.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
@@ -101,7 +102,7 @@ public class DataFileFooterConverter {
         dataFileFooter.setBlockletIndex(blockletIndex);
         dataFileFooter.setColumnInTable(columnSchemaList);
         dataFileFooter.setNumberOfRows(readBlockIndexInfo.getNum_rows());
-        dataFileFooter.setTableBlockInfo(tableBlockInfo);
+        dataFileFooter.setBlockInfo(new BlockInfo(tableBlockInfo));
         dataFileFooter.setSegmentInfo(segmentInfo);
         dataFileFooters.add(dataFileFooter);
       }


[37/50] [abbrv] incubator-carbondata git commit: Handle all dictionary exception more properly This closes #100

Posted by ra...@apache.org.
Handle all dictionary exception more properly This closes #100


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

Branch: refs/heads/branch-0.1
Commit: b2e47777c54e512daacb2cabf6a34c1bea9e2160
Parents: eedfe59
Author: ravipesala <ra...@gmail.com>
Authored: Tue Aug 30 13:12:25 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:30:34 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[24/50] [abbrv] incubator-carbondata git commit: show cost as mills instead of seconds

Posted by ra...@apache.org.
show cost as mills instead of seconds


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

Branch: refs/heads/branch-0.1
Commit: 956d9889d1e8c128e1a19dab19607fb303aeb877
Parents: e34ccde
Author: foryou2030 <fo...@126.com>
Authored: Wed Sep 7 21:58:18 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:58:21 2016 +0530

----------------------------------------------------------------------
 .../DriverQueryStatisticsRecorder.java             | 17 ++++++++---------
 .../carbon/querystatistics/QueryStatistic.java     |  4 ++--
 .../apache/carbondata/core/util/CarbonUtil.java    | 13 -------------
 3 files changed, 10 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/956d9889/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
index 534852f..c756b39 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
@@ -27,7 +27,6 @@ import java.util.concurrent.ConcurrentHashMap;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 
-import static org.apache.carbondata.core.util.CarbonUtil.add;
 import static org.apache.carbondata.core.util.CarbonUtil.printLine;
 
 import org.apache.commons.lang3.StringUtils;
@@ -121,8 +120,8 @@ public class DriverQueryStatisticsRecorder {
     String load_meta_time = "";
     String block_allocation_time = "";
     String block_identification_time = "";
-    Double driver_part_time_tmp = 0.0;
-    Double driver_part_time_tmp2 = 0.0;
+    long driver_part_time_tmp = 0L;
+    long driver_part_time_tmp2 = 0L;
     String splitChar = " ";
     try {
       // get statistic time from the QueryStatistic
@@ -130,21 +129,21 @@ public class DriverQueryStatisticsRecorder {
         switch (statistic.getMessage()) {
           case QueryStatisticsConstants.SQL_PARSE:
             sql_parse_time += statistic.getTimeTaken() + splitChar;
-            driver_part_time_tmp = add(driver_part_time_tmp, statistic.getTimeTaken());
+            driver_part_time_tmp += statistic.getTimeTaken();
             break;
           case QueryStatisticsConstants.LOAD_META:
             load_meta_time += statistic.getTimeTaken() + splitChar;
-            driver_part_time_tmp = add(driver_part_time_tmp, statistic.getTimeTaken());
+            driver_part_time_tmp += statistic.getTimeTaken();
             break;
           case QueryStatisticsConstants.BLOCK_ALLOCATION:
             block_allocation_time += statistic.getTimeTaken() + splitChar;
-            driver_part_time_tmp = add(driver_part_time_tmp, statistic.getTimeTaken());
-            driver_part_time_tmp2 = add(driver_part_time_tmp2, statistic.getTimeTaken());
+            driver_part_time_tmp += statistic.getTimeTaken();
+            driver_part_time_tmp2 += statistic.getTimeTaken();
             break;
           case QueryStatisticsConstants.BLOCK_IDENTIFICATION:
             block_identification_time += statistic.getTimeTaken() + splitChar;
-            driver_part_time_tmp = add(driver_part_time_tmp, statistic.getTimeTaken());
-            driver_part_time_tmp2 = add(driver_part_time_tmp2, statistic.getTimeTaken());
+            driver_part_time_tmp += statistic.getTimeTaken();
+            driver_part_time_tmp2 += statistic.getTimeTaken();
             break;
           default:
             break;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/956d9889/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java
index 3e44949..939b819 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java
@@ -98,8 +98,8 @@ public class QueryStatistic implements Serializable {
     return this.message;
   }
 
-  public double getTimeTaken() {
-    return (double)this.timeTaken/1000;
+  public long getTimeTaken() {
+    return  this.timeTaken;
   }
 
   public long getCount() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/956d9889/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 c98456e..82c515c 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
@@ -28,7 +28,6 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
-import java.math.BigDecimal;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.charset.Charset;
@@ -1432,17 +1431,5 @@ public final class CarbonUtil {
     return builder.toString();
   }
 
-  /**
-   * Below method will for double plus double
-   *
-   * @param v1
-   * @param v2
-   */
-  public static double add(double v1, double v2)
-  {
-    BigDecimal b1 = new BigDecimal(Double.toString(v1));
-    BigDecimal b2 = new BigDecimal(Double.toString(v2));
-    return  b1.add(b2).doubleValue();
-  }
 }
 


[13/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-222] No dictionary column offset problem. This closes #137

Posted by ra...@apache.org.
[CARBONDATA-222] No dictionary column offset problem. This closes #137


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

Branch: refs/heads/branch-0.1
Commit: 6411fde0aac03cdf60aa5ef8cd67dfe144bf9cc2
Parents: 32ce791
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Thu Sep 8 14:17:09 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:39:03 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[23/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-204]Clear queryStatisticsMap when timeout for query statistic This closes #123

Posted by ra...@apache.org.
[CARBONDATA-204]Clear queryStatisticsMap when timeout for query statistic This closes #123


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

Branch: refs/heads/branch-0.1
Commit: e34ccdef850dc4bd79a1f340b6eaaf44b1a70c56
Parents: 60fde41
Author: Vimal Das Kammath <vk...@VIMALDK-M01.vmware.com>
Authored: Tue Sep 6 12:15:06 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:57:30 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[42/50] [abbrv] incubator-carbondata git commit: inspection

Posted by ra...@apache.org.
inspection

code inspection optiminization

style

style


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

Branch: refs/heads/branch-0.1
Commit: 5928bb92fb3b3940d4a5e520f74b596e674228b1
Parents: 4a11f07
Author: Zhangshunyu <zh...@huawei.com>
Authored: Sun Sep 18 11:00:35 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:34:27 2016 +0530

----------------------------------------------------------------------
 .../compression/type/UnCompressDefaultLong.java |  4 +---
 .../spark/rdd/CarbonDataLoadRDD.scala           |  2 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  2 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  6 ++---
 .../carbondata/spark/util/CommonUtil.scala      |  8 +++----
 .../spark/util/GlobalDictionaryUtil.scala       |  2 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  | 24 ++++++++++----------
 .../execution/command/carbonTableSchema.scala   |  4 ++--
 .../processing/mdkeygen/MDKeyGenStep.java       |  8 ++-----
 .../util/CarbonDataProcessorUtil.java           |  5 ++--
 10 files changed, 29 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressDefaultLong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressDefaultLong.java b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressDefaultLong.java
index b30932c..a4d3b96 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressDefaultLong.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastorage/store/compression/type/UnCompressDefaultLong.java
@@ -41,9 +41,7 @@ public class UnCompressDefaultLong extends UnCompressNoneLong {
   @Override public CarbonReadDataHolder getValues(int decimal, Object maxValueObject) {
     CarbonReadDataHolder dataHolder = new CarbonReadDataHolder();
     long[] vals = new long[value.length];
-    for (int i = 0; i < vals.length; i++) {
-      vals[i] = value[i];
-    }
+    System.arraycopy(value, 0, vals, 0, vals.length);
     dataHolder.setReadableLongValues(vals);
     return dataHolder;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
index 60a2d00..86d58a5 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataLoadRDD.scala
@@ -175,7 +175,7 @@ class CarbonDataLoadRDD[K, V](
 
         if(carbonUseLocalDir.equalsIgnoreCase("true")) {
           val storeLocations = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
-          if (null != storeLocations && storeLocations.length > 0) {
+          if (null != storeLocations && storeLocations.nonEmpty) {
             storeLocation = storeLocations(Random.nextInt(storeLocations.length))
           }
           if (storeLocation == null) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/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 31cc8ac..3118d3f 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
@@ -696,7 +696,7 @@ object CarbonDataRDDFactory extends Logging {
         )
         var storeLocation = ""
         val configuredStore = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
-        if (null != configuredStore && configuredStore.length > 0) {
+        if (null != configuredStore && configuredStore.nonEmpty) {
           storeLocation = configuredStore(Random.nextInt(configuredStore.length))
         }
         if (storeLocation == null) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index 54d7539..8136b35 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -319,9 +319,9 @@ class CarbonMergerRDD[K, V](
           .add(new NodeInfo(blocksPerNode.getTaskId, blocksPerNode.getTableBlockInfoList.size))
        })
       if (list.size() != 0) {
-           result.add(new CarbonSparkPartition(id, i, Seq(entry._1).toArray, list))
-           i += 1
-         }
+        result.add(new CarbonSparkPartition(id, i, Seq(entry._1).toArray, list))
+        i += 1
+      }
     }
 
     // print the node info along with task and number of blocks for the task.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 3d85c1a..67f06e3 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -34,7 +34,7 @@ object CommonUtil {
       if (noDictionaryDims.contains(x)) {
         throw new MalformedCarbonCommandException(
           "Column group is not supported for no dictionary columns:" + x)
-      } else if (msrs.filter { msr => msr.column.equals(x) }.size > 0) {
+      } else if (msrs.filter { msr => msr.column.equals(x) }.nonEmpty) {
         // if column is measure
         throw new MalformedCarbonCommandException("Column group is not supported for measures:" + x)
       } else if (foundIndExistingColGrp(x)) {
@@ -47,7 +47,7 @@ object CommonUtil {
           "Column group doesn't support Timestamp datatype:" + x)
       }
       // if invalid column is present
-      else if (dims.filter { dim => dim.column.equalsIgnoreCase(x) }.size == 0) {
+      else if (dims.filter { dim => dim.column.equalsIgnoreCase(x) }.isEmpty) {
         throw new MalformedCarbonCommandException(
           "column in column group is not a valid column :" + x
         )
@@ -69,7 +69,7 @@ object CommonUtil {
   def isTimeStampColumn(colName: String, dims: Seq[Field]): Boolean = {
     dims.foreach { dim =>
       if (dim.column.equalsIgnoreCase(colName)) {
-        if (None != dim.dataType && null != dim.dataType.get &&
+        if (dim.dataType.isDefined && null != dim.dataType.get &&
             "timestamp".equalsIgnoreCase(dim.dataType.get)) {
           return true
         }
@@ -80,7 +80,7 @@ object CommonUtil {
 
   def isComplex(colName: String, dims: Seq[Field]): Boolean = {
     dims.foreach { x =>
-      if (None != x.children && null != x.children.get && x.children.get.size > 0) {
+      if (x.children.isDefined && null != x.children.get && x.children.get.nonEmpty) {
         val children = x.children.get
         if (x.column.equals(colName)) {
           return true

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index 02b70d0..818aa4a 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -596,7 +596,7 @@ object GlobalDictionaryUtil extends Logging {
     var columnName: String = ""
     var value: String = ""
     // such as "," , "", throw ex
-    if (tokens.size == 0) {
+    if (tokens.isEmpty) {
       logError("Read a bad dictionary record: " + x)
       accum += 1
     } else if (tokens.size == 1) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index 5675603..539c302 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -363,7 +363,7 @@ class CarbonSqlParser()
                 val dupColsGrp = cols.asScala.groupBy(x => x.getName) filter {
                   case (_, colList) => colList.size > 1
                 }
-                if (dupColsGrp.size > 0) {
+                if (dupColsGrp.nonEmpty) {
                   var columnName: String = ""
                   dupColsGrp.toSeq.foreach(columnName += _._1 + ", ")
                   columnName = columnName.substring(0, columnName.lastIndexOf(", "))
@@ -454,7 +454,7 @@ class CarbonSqlParser()
         catch {
           case ce: MalformedCarbonCommandException =>
             val message = if (tableName.isEmpty) "Create table command failed. "
-            else if (!dbName.isDefined) s"Create table command failed for $tableName. "
+            else if (dbName.isEmpty) s"Create table command failed for $tableName. "
             else s"Create table command failed for ${dbName.get}.$tableName. "
             LOGGER.audit(message + ce.getMessage)
             throw ce
@@ -515,7 +515,7 @@ class CarbonSqlParser()
 
     val (dims: Seq[Field], noDictionaryDims: Seq[String]) = extractDimColsAndNoDictionaryFields(
       fields, tableProperties)
-    if (dims.length == 0) {
+    if (dims.isEmpty) {
       throw new MalformedCarbonCommandException(s"Table ${dbName.getOrElse(
         CarbonCommonConstants.DATABASE_DEFAULT_NAME)}.$tableName"
         + " can not be created without key columns. Please use DICTIONARY_INCLUDE or " +
@@ -554,7 +554,7 @@ class CarbonSqlParser()
       noDictionaryDims: Seq[String],
       msrs: Seq[Field],
       dims: Seq[Field]): Seq[String] = {
-    if (None != tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS)) {
+    if (tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS).isDefined) {
 
       var splittedColGrps: Seq[String] = Seq[String]()
       val nonSplitCols: String = tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS).get
@@ -603,7 +603,7 @@ class CarbonSqlParser()
       true
     }
     val colGrpNames: StringBuilder = StringBuilder.newBuilder
-    for (i <- 0 until colGrpFieldIndx.length) {
+    for (i <- colGrpFieldIndx.indices) {
       colGrpNames.append(dims(colGrpFieldIndx(i)).column)
       if (i < (colGrpFieldIndx.length - 1)) {
         colGrpNames.append(",")
@@ -629,11 +629,11 @@ class CarbonSqlParser()
     var partitionClass: String = ""
     var partitionCount: Int = 1
     var partitionColNames: Array[String] = Array[String]()
-    if (None != tableProperties.get(CarbonCommonConstants.PARTITIONCLASS)) {
+    if (tableProperties.get(CarbonCommonConstants.PARTITIONCLASS).isDefined) {
       partitionClass = tableProperties.get(CarbonCommonConstants.PARTITIONCLASS).get
     }
 
-    if (None != tableProperties.get(CarbonCommonConstants.PARTITIONCOUNT)) {
+    if (tableProperties.get(CarbonCommonConstants.PARTITIONCOUNT).isDefined) {
       try {
         partitionCount = tableProperties.get(CarbonCommonConstants.PARTITIONCOUNT).get.toInt
       } catch {
@@ -684,14 +684,14 @@ class CarbonSqlParser()
     colPropMap: java.util.HashMap[String, java.util.List[ColumnProperty]]) {
     val (tblPropKey, colProKey) = getKey(parentColumnName, columnName)
     val colProps = CommonUtil.getColumnProperties(tblPropKey, tableProperties)
-    if (None != colProps) {
+    if (colProps.isDefined) {
       colPropMap.put(colProKey, colProps.get)
     }
   }
 
   def getKey(parentColumnName: Option[String],
     columnName: String): (String, String) = {
-    if (None != parentColumnName) {
+    if (parentColumnName.isDefined) {
       if (columnName == "val") {
         (parentColumnName.get, parentColumnName.get + "." + columnName)
       } else {
@@ -881,13 +881,13 @@ class CarbonSqlParser()
     var dictExcludedCols: Array[String] = Array[String]()
 
     // get all included cols
-    if (None != tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE)) {
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).isDefined) {
       dictIncludedCols =
         tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).get.split(',').map(_.trim)
     }
 
     // get all excluded cols
-    if (None != tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE)) {
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).isDefined) {
       dictExcludedCols =
         tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
     }
@@ -1038,7 +1038,7 @@ class CarbonSqlParser()
       case (_, optionlist) => optionlist.size > 1
     }
     val duplicates = StringBuilder.newBuilder
-    if (duplicateOptions.size > 0) {
+    if (duplicateOptions.nonEmpty) {
       duplicateOptions.foreach(x => {
         duplicates.append(x._1)
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/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 2047872..e4a79ab 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
@@ -211,7 +211,7 @@ class TableNewProcessor(cm: tableModel, sqlContext: SQLContext) {
       encoders.add(Encoding.DIRECT_DICTIONARY)
     }
     val colPropMap = new java.util.HashMap[String, String]()
-    if (None != cm.colProps && null != cm.colProps.get.get(colName)) {
+    if (cm.colProps.isDefined && null != cm.colProps.get.get(colName)) {
       val colProps = cm.colProps.get.get(colName)
       colProps.asScala.foreach { x => colPropMap.put(x.key, x.value) }
     }
@@ -1071,7 +1071,7 @@ private[sql] case class LoadTable(
       carbonLoadModel.setCarbonDataLoadSchema(dataLoadSchema)
       var storeLocation = ""
       val configuredStore = CarbonLoaderUtil.getConfiguredLocalDirs(SparkEnv.get.conf)
-      if (null != configuredStore && configuredStore.length > 0) {
+      if (null != configuredStore && configuredStore.nonEmpty) {
         storeLocation = configuredStore(Random.nextInt(configuredStore.length))
       }
       if (storeLocation == null) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/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
index ada5cc8..1f883dc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/mdkeygen/MDKeyGenStep.java
@@ -302,9 +302,7 @@ public class MDKeyGenStep extends BaseStep {
 
     int simpleDimsCount = this.dimensionCount - meta.getComplexDimsCount();
     int[] simpleDimsLen = new int[simpleDimsCount];
-    for (int i = 0; i < simpleDimsCount; i++) {
-      simpleDimsLen[i] = dimLens[i];
-    }
+    System.arraycopy(dimLens, 0, simpleDimsLen, 0, simpleDimsCount);
 
     CarbonTable carbonTable = CarbonMetadata.getInstance()
         .getCarbonTable(meta.getDatabaseName() + CarbonCommonConstants.UNDERSCORE + tableName);
@@ -352,9 +350,7 @@ public class MDKeyGenStep extends BaseStep {
   private void initDataHandler() {
     int simpleDimsCount = this.dimensionCount - meta.getComplexDimsCount();
     int[] simpleDimsLen = new int[simpleDimsCount];
-    for (int i = 0; i < simpleDimsCount; i++) {
-      simpleDimsLen[i] = dimLens[i];
-    }
+    System.arraycopy(dimLens, 0, simpleDimsLen, 0, simpleDimsCount);
     CarbonDataFileAttributes carbonDataFileAttributes =
         new CarbonDataFileAttributes(meta.getTaskNo(), meta.getFactTimeStamp());
     initAggType();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5928bb92/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 9168556..680d730 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
@@ -231,8 +231,7 @@ public final class CarbonDataProcessorUtil {
           .append(CarbonCommonConstants.HASH_SPC_CHARACTER);
     }
     String loadNames =
-        builder.substring(0, builder.lastIndexOf(CarbonCommonConstants.HASH_SPC_CHARACTER))
-            .toString();
+        builder.substring(0, builder.lastIndexOf(CarbonCommonConstants.HASH_SPC_CHARACTER));
     return loadNames;
   }
 
@@ -247,7 +246,7 @@ public final class CarbonDataProcessorUtil {
           .append(CarbonCommonConstants.HASH_SPC_CHARACTER);
     }
     String modOrDelTimesStamp =
-        builder.substring(0, builder.indexOf(CarbonCommonConstants.HASH_SPC_CHARACTER)).toString();
+        builder.substring(0, builder.indexOf(CarbonCommonConstants.HASH_SPC_CHARACTER));
     return modOrDelTimesStamp;
   }
 


[05/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-165] Support loading fact file with header for all dictionary This closes #82

Posted by ra...@apache.org.
[CARBONDATA-165] Support loading fact file with header for all dictionary This closes #82


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

Branch: refs/heads/branch-0.1
Commit: ce34e10c1fe99d46dfa6b11c41a31dfeb4895aed
Parents: 51e4c11
Author: ravipesala <ra...@gmail.com>
Authored: Tue Aug 30 16:38:29 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:29:46 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[17/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-222] Null filter member in 'NOT IN' filter behaviour, not compatible with hive. This closes #159

Posted by ra...@apache.org.
[CARBONDATA-222]  Null filter member in 'NOT IN' filter behaviour, not compatible with hive. This closes #159


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

Branch: refs/heads/branch-0.1
Commit: 595460cb75e07253df948bb3d69a942f5d63b029
Parents: 772382d
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Fri Sep 16 22:01:10 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:41:14 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[49/50] [abbrv] incubator-carbondata git commit: disabling the system compaction lock feature. and making the load ddl to wait for compaction to finish in the auto compaction case.

Posted by ra...@apache.org.
disabling the system compaction lock feature. and making the load ddl to wait for compaction to finish in the auto compaction case.


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

Branch: refs/heads/branch-0.1
Commit: fe274a9f58e353f925374cf5ec51484ef4b23ffe
Parents: b55e7d3
Author: ravikiran <ra...@gmail.com>
Authored: Sat Sep 17 15:18:22 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:52:31 2016 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  3 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        | 64 ++++++++++----------
 .../CompactionSystemLockFeatureTest.scala       |  2 +
 3 files changed, 37 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fe274a9f/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 41d6ebf..1d60ee9 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
@@ -911,6 +911,7 @@ public final class CarbonCommonConstants {
   public static String majorCompactionRequiredFile = "compactionRequired_major";
 
   /**
+   * @Deprecated : This property has been deprecated.
    * Property for enabling system level compaction lock.1 compaction can run at once.
    */
   public static String ENABLE_CONCURRENT_COMPACTION =
@@ -920,7 +921,7 @@ public final class CarbonCommonConstants {
    * Default value of Property for enabling system level compaction lock.1 compaction can run
    * at once.
    */
-  public static String DEFAULT_ENABLE_CONCURRENT_COMPACTION = "false";
+  public static String DEFAULT_ENABLE_CONCURRENT_COMPACTION = "true";
 
   /**
    * Compaction system level lock folder.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fe274a9f/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 964c955..3e0388f 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
@@ -592,14 +592,9 @@ object CarbonDataRDDFactory extends Logging {
           }
         }
       }
-      if(compactionModel.isDDLTrigger) {
-        // making this an blocking call for DDL
-        compactionThread.run()
-      }
-      else {
-        // non blocking call in case of auto compaction.
-        compactionThread.start()
-      }
+    // calling the run method of a thread to make the call as blocking call.
+    // in the future we may make this as concurrent.
+    compactionThread.run()
   }
 
   def prepareCarbonLoadModel(hdfsStoreLocation: String,
@@ -715,6 +710,7 @@ object CarbonDataRDDFactory extends Logging {
               case e : Exception =>
                 logger.error("Exception in start compaction thread. " + e.getMessage)
                 lock.unlock()
+                throw e
             }
           }
           else {
@@ -786,9 +782,6 @@ object CarbonDataRDDFactory extends Logging {
       val schemaLastUpdatedTime = CarbonEnv.getInstance(sqlContext).carbonCatalog
         .getSchemaLastUpdatedTime(carbonLoadModel.getDatabaseName, carbonLoadModel.getTableName)
 
-      // compaction handling
-      handleSegmentMerging(tableCreationTime)
-
       // get partition way from configuration
       // val isTableSplitPartition = CarbonProperties.getInstance().getProperty(
       // CarbonCommonConstants.TABLE_SPLIT_PARTITION,
@@ -999,28 +992,37 @@ object CarbonDataRDDFactory extends Logging {
         logWarning("Cannot write load metadata file as data load failed")
         throw new Exception(errorMessage)
       } else {
-        val metadataDetails = status(0)._2
-        if (!isAgg) {
-          val status = CarbonLoaderUtil
-            .recordLoadMetadata(currentLoadCount,
-              metadataDetails,
-              carbonLoadModel,
-              loadStatus,
-              loadStartTime
-            )
-          if (!status) {
-            val errorMessage = "Dataload failed due to failure in table status updation."
-            logger.audit("Data load is failed for " +
-              s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
-            logger.error("Dataload failed due to failure in table status updation.")
-            throw new Exception(errorMessage)
+          val metadataDetails = status(0)._2
+          if (!isAgg) {
+            val status = CarbonLoaderUtil
+              .recordLoadMetadata(currentLoadCount,
+                metadataDetails,
+                carbonLoadModel,
+                loadStatus,
+                loadStartTime
+              )
+            if (!status) {
+              val errorMessage = "Dataload failed due to failure in table status updation."
+              logger.audit("Data load is failed for " +
+                           s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
+              logger.error("Dataload failed due to failure in table status updation.")
+              throw new Exception(errorMessage)
+            }
+          } else if (!carbonLoadModel.isRetentionRequest) {
+            // TODO : Handle it
+            logInfo("********Database updated**********")
           }
-        } else if (!carbonLoadModel.isRetentionRequest) {
-          // TODO : Handle it
-          logInfo("********Database updated**********")
+          logger.audit("Data load is successful for " +
+                       s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
+        try {
+          // compaction handling
+          handleSegmentMerging(tableCreationTime)
+        }
+        catch {
+          case e: Exception =>
+            throw new Exception(
+              "Dataload is success. Auto-Compaction has failed. Please check logs.")
         }
-        logger.audit("Data load is successful for " +
-          s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fe274a9f/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala
index d9e1349..a040550 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala
@@ -43,6 +43,8 @@ class CompactionSystemLockFeatureTest extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists  table2")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "mm/dd/yyyy")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.ENABLE_CONCURRENT_COMPACTION, "false")
     sql(
       "CREATE TABLE IF NOT EXISTS table1 (country String, ID Int, date Timestamp, name " +
         "String, " +


[41/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-202] Handled exception thrown in beeline for all dictionary This closes #122

Posted by ra...@apache.org.
[CARBONDATA-202] Handled exception thrown in beeline for all dictionary This closes #122


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

Branch: refs/heads/branch-0.1
Commit: 4a11f0716b0c6c981c9147c37977616d019570c3
Parents: 50b3746
Author: chenliang613 <ch...@apache.org>
Authored: Sat Sep 3 16:19:02 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:33:39 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[11/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-229] exception thrown while writing sort index file. This closes #143

Posted by ra...@apache.org.
[CARBONDATA-229] exception thrown while writing sort index file. This closes #143


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

Branch: refs/heads/branch-0.1
Commit: 008bee1e477bae1449d22a38b1e52e073938e229
Parents: ad14039
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Thu Sep 8 14:50:57 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:35:30 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[08/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-227] In block distribution parralelism is decided initially and not re initialized after requesting new executors. Due to this task per node initialization is getting wrong.

Posted by ra...@apache.org.
[CARBONDATA-227] In block distribution parralelism is decided initially and not re initialized after requesting new executors. Due to this task per node initialization is getting wrong.


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

Branch: refs/heads/branch-0.1
Commit: acb1d979e4f61bf727f69ce553f0f9cad0954ae8
Parents: e1f34cc
Author: mohammadshahidkhan <mo...@gmail.com>
Authored: Thu Sep 8 10:37:49 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:34:05 2016 +0530

----------------------------------------------------------------------
 .../apache/carbondata/spark/rdd/CarbonMergerRDD.scala    |  4 ++--
 .../org/apache/carbondata/spark/rdd/CarbonScanRDD.scala  | 11 ++++++++---
 2 files changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/acb1d979/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
index f160fd9..54d7539 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -55,7 +55,6 @@ class CarbonMergerRDD[K, V](
   confExecutorsTemp: String)
   extends RDD[(K, V)](sc, Nil) with Logging {
 
-  val defaultParallelism = sc.defaultParallelism
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
   sc.setLocalProperty("spark.job.interruptOnCancel", "true")
 
@@ -228,6 +227,7 @@ class CarbonMergerRDD[K, V](
     )
     val (carbonInputFormat: CarbonInputFormat[Array[Object]], job: Job) =
       QueryPlanUtil.createCarbonInputFormat(absoluteTableIdentifier)
+    var defaultParallelism = sparkContext.defaultParallelism
     val result = new util.ArrayList[Partition](defaultParallelism)
 
     // mapping of the node and block list.
@@ -299,7 +299,7 @@ class CarbonMergerRDD[K, V](
       maxTimes = maxTimes - 1
     }
     logInfo("Time taken to wait for executor allocation is =" + ((30 - maxTimes) * 500) + "millis")
-
+    defaultParallelism = sparkContext.defaultParallelism
     var i = 0
 
     val nodeTaskBlocksMap: util.Map[String, util.List[NodeInfo]] = new util.HashMap[String, util

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/acb1d979/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 5f50644..497d9f8 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -73,9 +73,9 @@ class CarbonScanRDD[V: ClassTag](
     baseStoreLocation: String)
   extends RDD[V](sc, Nil) with Logging {
 
-  val defaultParallelism = sc.defaultParallelism
 
   override def getPartitions: Array[Partition] = {
+    var defaultParallelism = sparkContext.defaultParallelism
     val statisticRecorder = CarbonTimeStatisticsFactory.getQueryStatisticsRecorderInstance()
     val (carbonInputFormat: CarbonInputFormat[Array[Object]], job: Job) =
       QueryPlanUtil.createCarbonInputFormat(queryModel.getAbsoluteTableIdentifier)
@@ -110,13 +110,18 @@ class CarbonScanRDD[V: ClassTag](
           new BlockletInfos(inputSplit.getNumberOfBlocklets, 0, inputSplit.getNumberOfBlocklets)
         )
       )
+      var activeNodes = Array[String]()
+      if(blockListTemp.nonEmpty) {
+         activeNodes = DistributionUtil
+          .ensureExecutorsAndGetNodeList(blockListTemp.toArray, sparkContext)
+      }
+      defaultParallelism = sparkContext.defaultParallelism
       val blockList = CarbonLoaderUtil.
         distributeBlockLets(blockListTemp.asJava, defaultParallelism).asScala
+
       if (blockList.nonEmpty) {
         var statistic = new QueryStatistic()
         // group blocks to nodes, tasks
-        val activeNodes = DistributionUtil
-          .ensureExecutorsAndGetNodeList(blockList.toArray, sparkContext)
         val nodeBlockMapping =
           CarbonLoaderUtil.nodeBlockTaskMapping(blockList.asJava, -1, defaultParallelism,
             activeNodes.toList.asJava


[47/50] [abbrv] incubator-carbondata git commit: Remove binary file generate the binary file on fly while running the test case.

Posted by ra...@apache.org.
Remove binary file generate the binary file on fly while running the test case.

Fixed comments


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

Branch: refs/heads/branch-0.1
Commit: 263ac7a3d27988bc199a46cd2047451b7aade8a2
Parents: a01917b
Author: ravipesala <ra...@gmail.com>
Authored: Tue Sep 20 18:32:01 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:37:37 2016 +0530

----------------------------------------------------------------------
 .../carbon/datastore/BlockIndexStoreTest.java   | 204 -------
 .../resources/part-0-0-1466029397000.carbondata | Bin 25413 -> 0 bytes
 .../carbon/datastore/BlockIndexStoreTest.java   | 225 ++++++++
 .../carbondata/test/util/StoreCreator.java      | 564 +++++++++++++++++++
 4 files changed, 789 insertions(+), 204 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/263ac7a3/core/src/test/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStoreTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStoreTest.java b/core/src/test/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStoreTest.java
deleted file mode 100644
index 3ff0f87..0000000
--- a/core/src/test/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStoreTest.java
+++ /dev/null
@@ -1,204 +0,0 @@
-///*
-// * Licensed to the Apache Software Foundation (ASF) under one
-// * or more contributor license agreements.  See the NOTICE file
-// * distributed with this work for additional information
-// * regarding copyright ownership.  The ASF licenses this file
-// * to you under the Apache License, Version 2.0 (the
-// * "License"); you may not use this file except in compliance
-// * with the License.  You may obtain a copy of the License at
-// *
-// *    http://www.apache.org/licenses/LICENSE-2.0
-// *
-// * Unless required by applicable law or agreed to in writing,
-// * software distributed under the License is distributed on an
-// * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// * KIND, either express or implied.  See the License for the
-// * specific language governing permissions and limitations
-// * under the License.
-// */
-//package org.apache.carbondata.core.carbon.datastore;
-//
-//import java.io.File;
-//import java.io.IOException;
-//import java.util.Arrays;
-//import java.util.List;
-//import java.util.concurrent.Callable;
-//import java.util.concurrent.ExecutorService;
-//import java.util.concurrent.Executors;
-//import java.util.concurrent.TimeUnit;
-//
-//import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
-//import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
-//import org.apache.carbondata.core.carbon.datastore.block.AbstractIndex;
-//import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
-//import org.apache.carbondata.core.carbon.datastore.exception.IndexBuilderException;
-//
-//import junit.framework.TestCase;
-//import org.junit.BeforeClass;
-//import org.junit.Test;
-//
-//public class BlockIndexStoreTest extends TestCase {
-//
-//  private BlockIndexStore indexStore;
-//
-//  @BeforeClass public void setUp() {
-//    indexStore = BlockIndexStore.getInstance();
-//  }
-//
-//  @Test public void testloadAndGetTaskIdToSegmentsMapForSingleSegment() throws IOException {
-//    String canonicalPath =
-//        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
-//    File file = new File(canonicalPath + "/src/test/resources/part-0-0-1466029397000.carbondata");
-//    TableBlockInfo info =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length());
-//    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
-//        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-//    try {
-//      List<AbstractIndex> loadAndGetBlocks = indexStore
-//          .loadAndGetBlocks(Arrays.asList(new TableBlockInfo[] { info }), absoluteTableIdentifier);
-//      assertTrue(loadAndGetBlocks.size() == 1);
-//    } catch (IndexBuilderException e) {
-//      assertTrue(false);
-//    }
-//    indexStore.clear(absoluteTableIdentifier);
-//  }
-//
-//  @Test public void testloadAndGetTaskIdToSegmentsMapForSameBlockLoadedConcurrently()
-//      throws IOException {
-//    String canonicalPath =
-//        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
-//    File file = new File(canonicalPath + "/src/test/resources/part-0-0-1466029397000.carbondata");
-//    TableBlockInfo info =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length());
-//    TableBlockInfo info1 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length());
-//
-//    TableBlockInfo info2 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length());
-//    TableBlockInfo info3 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length());
-//    TableBlockInfo info4 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length());
-//
-//    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
-//        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-//    ExecutorService executor = Executors.newFixedThreadPool(3);
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
-//    executor.submit(
-//        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
-//    executor.submit(
-//        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
-//    executor.shutdown();
-//    try {
-//      executor.awaitTermination(1, TimeUnit.DAYS);
-//    } catch (InterruptedException e) {
-//      e.printStackTrace();
-//    }
-//
-//    try {
-//      List<AbstractIndex> loadAndGetBlocks = indexStore.loadAndGetBlocks(
-//          Arrays.asList(new TableBlockInfo[] { info, info1, info2, info3, info4 }),
-//          absoluteTableIdentifier);
-//      assertTrue(loadAndGetBlocks.size() == 5);
-//    } catch (IndexBuilderException e) {
-//      assertTrue(false);
-//    }
-//    indexStore.clear(absoluteTableIdentifier);
-//  }
-//
-//  @Test public void testloadAndGetTaskIdToSegmentsMapForDifferentSegmentLoadedConcurrently()
-//      throws IOException {
-//    String canonicalPath =
-//        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
-//    File file = new File(canonicalPath + "/src/test/resources/part-0-0-1466029397000.carbondata");
-//    TableBlockInfo info =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length());
-//    TableBlockInfo info1 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-//            file.length());
-//
-//    TableBlockInfo info2 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length());
-//    TableBlockInfo info3 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length());
-//    TableBlockInfo info4 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-//            file.length());
-//
-//    TableBlockInfo info5 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "2", new String[] { "loclhost" },
-//            file.length());
-//    TableBlockInfo info6 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "2", new String[] { "loclhost" },
-//            file.length());
-//
-//    TableBlockInfo info7 =
-//        new TableBlockInfo(file.getAbsolutePath(), 0, "3", new String[] { "loclhost" },
-//            file.length());
-//
-//    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
-//    AbsoluteTableIdentifier absoluteTableIdentifier =
-//        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-//    ExecutorService executor = Executors.newFixedThreadPool(3);
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
-//        absoluteTableIdentifier));
-//    executor.submit(
-//        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
-//            absoluteTableIdentifier));
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info5, info6 }),
-//        absoluteTableIdentifier));
-//    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info7 }),
-//        absoluteTableIdentifier));
-//
-//    executor.shutdown();
-//    try {
-//      executor.awaitTermination(1, TimeUnit.DAYS);
-//    } catch (InterruptedException e) {
-//      // TODO Auto-generated catch block
-//      e.printStackTrace();
-//    }
-//    try {
-//      List<AbstractIndex> loadAndGetBlocks = indexStore.loadAndGetBlocks(Arrays
-//              .asList(new TableBlockInfo[] { info, info1, info2, info3, info4, info5, info6, info7 }),
-//          absoluteTableIdentifier);
-//      assertTrue(loadAndGetBlocks.size() == 8);
-//    } catch (IndexBuilderException e) {
-//      assertTrue(false);
-//    }
-//    indexStore.clear(absoluteTableIdentifier);
-//  }
-//
-//  private class BlockLoaderThread implements Callable<Void> {
-//    private List<TableBlockInfo> tableBlockInfoList;
-//    private AbsoluteTableIdentifier absoluteTableIdentifier;
-//
-//    public BlockLoaderThread(List<TableBlockInfo> tableBlockInfoList,
-//        AbsoluteTableIdentifier absoluteTableIdentifier) {
-//      // TODO Auto-generated constructor stub
-//      this.tableBlockInfoList = tableBlockInfoList;
-//      this.absoluteTableIdentifier = absoluteTableIdentifier;
-//    }
-//
-//    @Override public Void call() throws Exception {
-//      indexStore.loadAndGetBlocks(tableBlockInfoList, absoluteTableIdentifier);
-//      return null;
-//    }
-//
-//  }
-//}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/263ac7a3/core/src/test/resources/part-0-0-1466029397000.carbondata
----------------------------------------------------------------------
diff --git a/core/src/test/resources/part-0-0-1466029397000.carbondata b/core/src/test/resources/part-0-0-1466029397000.carbondata
deleted file mode 100644
index e8c748d..0000000
Binary files a/core/src/test/resources/part-0-0-1466029397000.carbondata and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/263ac7a3/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
new file mode 100644
index 0000000..b13a67d
--- /dev/null
+++ b/processing/src/test/java/org/apache/carbondata/carbon/datastore/BlockIndexStoreTest.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.carbon.datastore;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
+import org.apache.carbondata.core.carbon.datastore.BlockIndexStore;
+import org.apache.carbondata.core.carbon.datastore.block.AbstractIndex;
+import org.apache.carbondata.core.carbon.datastore.block.TableBlockInfo;
+import org.apache.carbondata.core.carbon.datastore.exception.IndexBuilderException;
+import org.apache.carbondata.test.util.StoreCreator;
+
+import junit.framework.TestCase;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class BlockIndexStoreTest extends TestCase {
+
+  private BlockIndexStore indexStore;
+
+  @BeforeClass public void setUp() {
+    StoreCreator.createCarbonStore();
+    indexStore = BlockIndexStore.getInstance();
+  }
+
+  @Test public void testloadAndGetTaskIdToSegmentsMapForSingleSegment() throws IOException {
+    String canonicalPath =
+        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
+    File file = getPartFile();
+    TableBlockInfo info =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
+            file.length());
+    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
+    AbsoluteTableIdentifier absoluteTableIdentifier =
+        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
+    try {
+      List<AbstractIndex> loadAndGetBlocks = indexStore
+          .loadAndGetBlocks(Arrays.asList(new TableBlockInfo[] { info }), absoluteTableIdentifier);
+      assertTrue(loadAndGetBlocks.size() == 1);
+    } catch (IndexBuilderException e) {
+      assertTrue(false);
+    }
+    indexStore.clear(absoluteTableIdentifier);
+  }
+
+  @Test public void testloadAndGetTaskIdToSegmentsMapForSameBlockLoadedConcurrently()
+      throws IOException {
+    String canonicalPath =
+        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
+    File file = getPartFile();
+    TableBlockInfo info =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
+            file.length());
+    TableBlockInfo info1 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
+            file.length());
+
+    TableBlockInfo info2 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
+            file.length());
+    TableBlockInfo info3 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
+            file.length());
+    TableBlockInfo info4 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
+            file.length());
+
+    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
+    AbsoluteTableIdentifier absoluteTableIdentifier =
+        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
+    ExecutorService executor = Executors.newFixedThreadPool(3);
+    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
+        absoluteTableIdentifier));
+    executor.submit(
+        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
+            absoluteTableIdentifier));
+    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
+        absoluteTableIdentifier));
+    executor.submit(
+        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
+            absoluteTableIdentifier));
+    executor.shutdown();
+    try {
+      executor.awaitTermination(1, TimeUnit.DAYS);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
+    }
+
+    try {
+      List<AbstractIndex> loadAndGetBlocks = indexStore.loadAndGetBlocks(
+          Arrays.asList(new TableBlockInfo[] { info, info1, info2, info3, info4 }),
+          absoluteTableIdentifier);
+      assertTrue(loadAndGetBlocks.size() == 5);
+    } catch (IndexBuilderException e) {
+      assertTrue(false);
+    }
+    indexStore.clear(absoluteTableIdentifier);
+  }
+
+  @Test public void testloadAndGetTaskIdToSegmentsMapForDifferentSegmentLoadedConcurrently()
+      throws IOException {
+    String canonicalPath =
+        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
+    File file = getPartFile();
+    TableBlockInfo info =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
+            file.length());
+    TableBlockInfo info1 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
+            file.length());
+
+    TableBlockInfo info2 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
+            file.length());
+    TableBlockInfo info3 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
+            file.length());
+    TableBlockInfo info4 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
+            file.length());
+
+    TableBlockInfo info5 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "2", new String[] { "loclhost" },
+            file.length());
+    TableBlockInfo info6 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "2", new String[] { "loclhost" },
+            file.length());
+
+    TableBlockInfo info7 =
+        new TableBlockInfo(file.getAbsolutePath(), 0, "3", new String[] { "loclhost" },
+            file.length());
+
+    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
+    AbsoluteTableIdentifier absoluteTableIdentifier =
+        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
+    ExecutorService executor = Executors.newFixedThreadPool(3);
+    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info, info1 }),
+        absoluteTableIdentifier));
+    executor.submit(
+        new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info2, info3, info4 }),
+            absoluteTableIdentifier));
+    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info5, info6 }),
+        absoluteTableIdentifier));
+    executor.submit(new BlockLoaderThread(Arrays.asList(new TableBlockInfo[] { info7 }),
+        absoluteTableIdentifier));
+
+    executor.shutdown();
+    try {
+      executor.awaitTermination(1, TimeUnit.DAYS);
+    } catch (InterruptedException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    }
+    try {
+      List<AbstractIndex> loadAndGetBlocks = indexStore.loadAndGetBlocks(Arrays
+              .asList(new TableBlockInfo[] { info, info1, info2, info3, info4, info5, info6, info7 }),
+          absoluteTableIdentifier);
+      assertTrue(loadAndGetBlocks.size() == 8);
+    } catch (IndexBuilderException e) {
+      assertTrue(false);
+    }
+    indexStore.clear(absoluteTableIdentifier);
+  }
+
+  private class BlockLoaderThread implements Callable<Void> {
+    private List<TableBlockInfo> tableBlockInfoList;
+    private AbsoluteTableIdentifier absoluteTableIdentifier;
+
+    public BlockLoaderThread(List<TableBlockInfo> tableBlockInfoList,
+        AbsoluteTableIdentifier absoluteTableIdentifier) {
+      // TODO Auto-generated constructor stub
+      this.tableBlockInfoList = tableBlockInfoList;
+      this.absoluteTableIdentifier = absoluteTableIdentifier;
+    }
+
+    @Override public Void call() throws Exception {
+      indexStore.loadAndGetBlocks(tableBlockInfoList, absoluteTableIdentifier);
+      return null;
+    }
+
+  }
+
+  private static File getPartFile() {
+    String path = StoreCreator.getAbsoluteTableIdentifier().getStorePath() + "/" + StoreCreator
+        .getAbsoluteTableIdentifier().getCarbonTableIdentifier().getDatabaseName() + "/"
+        + StoreCreator.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName()
+        + "/Fact/Part0/Segment_0";
+    File file = new File(path);
+    File[] files = file.listFiles();
+    File part = null;
+    for (int i = 0; i < files.length; i++) {
+      if (files[i].getName().startsWith("part")) {
+        part = files[i];
+      }
+    }
+    return part;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/263ac7a3/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
new file mode 100644
index 0000000..2faabaf
--- /dev/null
+++ b/processing/src/test/java/org/apache/carbondata/test/util/StoreCreator.java
@@ -0,0 +1,564 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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 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.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.carbon.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.carbon.CarbonDataLoadSchema;
+import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
+import org.apache.carbondata.core.carbon.ColumnIdentifier;
+import org.apache.carbondata.core.carbon.metadata.CarbonMetadata;
+import org.apache.carbondata.core.carbon.metadata.converter.SchemaConverter;
+import org.apache.carbondata.core.carbon.metadata.converter.ThriftWrapperSchemaConverterImpl;
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.apache.carbondata.core.carbon.metadata.schema.SchemaEvolution;
+import org.apache.carbondata.core.carbon.metadata.schema.SchemaEvolutionEntry;
+import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.carbon.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.carbon.metadata.schema.table.TableSchema;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.carbon.path.CarbonStorePath;
+import org.apache.carbondata.core.carbon.path.CarbonTablePath;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
+import org.apache.carbondata.core.load.BlockDetails;
+import org.apache.carbondata.core.load.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.lcm.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.lcm.fileoperations.AtomicFileOperationsImpl;
+import org.apache.carbondata.lcm.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.carbondata.processing.util.CarbonDataProcessorUtil;
+
+import com.google.gson.Gson;
+
+/**
+ * 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 schmaModel = new DataProcessTaskStatus(databaseName, tableName);
+    schmaModel.setCsvFilePath(loadModel.getFactFilePath());
+    SchemaInfo info = new SchemaInfo();
+    BlockDetails blockDetails = new BlockDetails(loadModel.getFactFilePath(),
+        0, new File(loadModel.getFactFilePath()).length(), new String[] {"localhost"});
+    GraphGenerator.blockInfo.put("qwqwq", new BlockDetails[] { blockDetails });
+    schmaModel.setBlocksID("qwqwq");
+    schmaModel.setEscapeCharacter("\\");
+    schmaModel.setQuoteCharacter("\"");
+    schmaModel.setCommentCharacter("#");
+    info.setDatabaseName(databaseName);
+    info.setTableName(tableName);
+
+    generateGraph(schmaModel, info, loadModel.getTableName(), "0", loadModel.getSchema(), null,
+        loadModel.getLoadMetadataDetails());
+
+    DataGraphExecuter graphExecuter = new DataGraphExecuter(schmaModel);
+    graphExecuter
+        .executeGraph(graphPath, new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN),
+            info, "0", 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.setTimestamp(readCurrentTime());
+    loadMetadataDetails.setLoadStatus("SUCCESS");
+    loadMetadataDetails.setLoadName(String.valueOf(0));
+    loadMetadataDetails.setLoadStartTime(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 schmaModel
+   * @param info
+   * @param tableName
+   * @param partitionID
+   * @param schema
+   * @param factStoreLocation
+   * @param loadMetadataDetails
+   * @throws GraphGeneratorException
+   */
+  private static void generateGraph(IDataProcessStatus schmaModel, SchemaInfo info,
+      String tableName, String partitionID, CarbonDataLoadSchema schema, String factStoreLocation,
+      List<LoadMetadataDetails> loadMetadataDetails)
+      throws GraphGeneratorException {
+    DataLoadModel model = new DataLoadModel();
+    model.setCsvLoad(null != schmaModel.getCsvFilePath() || null != schmaModel.getFilesToProcess());
+    model.setSchemaInfo(info);
+    model.setTableName(schmaModel.getTableName());
+    model.setTaskNo("1");
+    model.setBlocksID(schmaModel.getBlocksID());
+    model.setFactTimeStamp(readCurrentTime());
+    model.setEscapeCharacter(schmaModel.getEscapeCharacter());
+    model.setQuoteCharacter(schmaModel.getQuoteCharacter());
+    model.setCommentCharacter(schmaModel.getCommentCharacter());
+    if (null != loadMetadataDetails && !loadMetadataDetails.isEmpty()) {
+      model.setLoadNames(
+          CarbonDataProcessorUtil.getLoadNameFromLoadMetaDataDetails(loadMetadataDetails));
+      model.setModificationOrDeletionTime(CarbonDataProcessorUtil
+          .getModificationOrDeletionTimesFromLoadMetadataDetails(loadMetadataDetails));
+    }
+    boolean hdfsReadMode =
+        schmaModel.getCsvFilePath() != null && schmaModel.getCsvFilePath().startsWith("hdfs:");
+    int allocate = null != schmaModel.getCsvFilePath() ? 1 : schmaModel.getFilesToProcess().size();
+    String outputLocation = CarbonProperties.getInstance()
+        .getProperty("store_output_location", "../carbon-store/system/carbon/etl");
+    GraphGenerator generator =
+        new GraphGenerator(model, hdfsReadMode, partitionID, factStoreLocation,
+            allocate, schema, "0", outputLocation);
+    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
+   *
+   * @author Administrator
+   */
+  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



[34/50] [abbrv] incubator-carbondata git commit: measure can not use columnDict

Posted by ra...@apache.org.
measure can not use columnDict


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

Branch: refs/heads/branch-0.1
Commit: 16533279bc45cb8eae22a9cccf891715d8c1bec9
Parents: 8f8933e
Author: Jay357089 <li...@huawei.com>
Authored: Wed Aug 24 10:08:01 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:28:43 2016 +0530

----------------------------------------------------------------------
 .../spark/util/GlobalDictionaryUtil.scala          |  7 ++++---
 .../util/ExternalColumnDictionaryTestCase.scala    | 17 +++++++++++++++++
 2 files changed, 21 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/16533279/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index 7e60320..cabedfd 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -499,9 +499,10 @@ object GlobalDictionaryUtil extends Logging {
     val preDictDimensionOption = dimensions.filter(
       _.getColName.equalsIgnoreCase(dimParent))
     if (preDictDimensionOption.length == 0) {
-      logError(s"No column $dimParent exists in ${table.getDatabaseName}.${table.getTableName}")
-      throw new DataLoadingException(s"No column $colName exists " +
-      s"in ${table.getDatabaseName}.${table.getTableName}")
+      logError(s"Column $dimParent is not a key column " +
+        s"in ${table.getDatabaseName}.${table.getTableName}")
+      throw new DataLoadingException(s"Column $dimParent is not a key column. " +
+        s"Only key column can be part of dictionary and used in COLUMNDICT option.")
     }
     val preDictDimension = preDictDimensionOption(0)
     if (preDictDimension.isComplex) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/16533279/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
index 4e013bb..44ca85e 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/util/ExternalColumnDictionaryTestCase.scala
@@ -21,6 +21,7 @@ package org.apache.carbondata.spark.util
 import java.io.File
 
 import org.apache.carbondata.core.carbon.CarbonDataLoadSchema
+import org.apache.carbondata.processing.etl.DataLoadingException
 import org.apache.carbondata.spark.exception.MalformedCarbonCommandException
 import org.apache.carbondata.spark.load.CarbonLoadModel
 import org.apache.spark.sql.{CarbonEnv, CarbonRelation}
@@ -217,6 +218,22 @@ class ExternalColumnDictionaryTestCase extends QueryTest with BeforeAndAfterAll
       case ex: MalformedCarbonCommandException =>
         assertResult(ex.getMessage)("Error: COLUMNDICT and ALL_DICTIONARY_PATH can not be used together " +
           "in options")
+      case _ => assert(false)
+    }
+  }
+
+  test("Measure can not use COLUMNDICT") {
+    try {
+      sql(s"""
+      LOAD DATA LOCAL INPATH "$complexFilePath1" INTO TABLE loadSqlTest
+      OPTIONS('COLUMNDICT'='gamePointId:$filePath')
+      """)
+      assert(false)
+    } catch {
+      case ex: DataLoadingException =>
+        assertResult(ex.getMessage)("Column gamePointId is not a key column. Only key column can be part " +
+          "of dictionary and used in COLUMNDICT option.")
+      case _ => assert(false)
     }
   }
 


[19/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-250] Throw exception and fail the data load if provided MAXCOLUMNS value is not proper. This closes #167

Posted by ra...@apache.org.
[CARBONDATA-250] Throw exception and fail the data load if provided MAXCOLUMNS value is not proper. This closes #167


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

Branch: refs/heads/branch-0.1
Commit: 87ba555322b0e4a73df8b6202f7418a4ae1ee5d2
Parents: 6861b7e
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sat Sep 17 22:08:11 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:45:03 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[38/50] [abbrv] incubator-carbondata git commit: give a proper error message for wrong delimiter or fileheader

Posted by ra...@apache.org.
give a proper error message for wrong delimiter or fileheader


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

Branch: refs/heads/branch-0.1
Commit: 0d14af2a043464c408a9ff917f3c0dd6280b3b87
Parents: b2e4777
Author: QiangCai <da...@gmail.com>
Authored: Fri Aug 26 10:57:53 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:31:42 2016 +0530

----------------------------------------------------------------------
 .../carbondata/spark/util/GlobalDictionaryUtil.scala      | 10 ++++++++--
 1 file changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/0d14af2a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index bdd8adc..022a5f8 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -737,6 +737,12 @@ object GlobalDictionaryUtil extends Logging {
           generatePredefinedColDictionary(colDictFilePath, table,
             dimensions, carbonLoadModel, sqlContext, hdfsLocation, dictfolderPath)
         }
+        if (headers.length > df.columns.length) {
+          val msg = "The number of columns in the file header do not match the number of " +
+            "columns in the data file; Either delimiter or fileheader provided is not correct"
+          logError(msg)
+          throw new DataLoadingException(msg)
+        }
         // use fact file to generate global dict
         val (requireDimension, requireColumnNames) = pruneDimensions(dimensions,
           headers, df.columns)
@@ -753,7 +759,7 @@ object GlobalDictionaryUtil extends Logging {
           // check result status
           checkStatus(carbonLoadModel, sqlContext, model, statusList)
         } else {
-          logInfo("have no column need to generate global dictionary in Fact file")
+          logInfo("No column found for generating global dictionary in source data files")
         }
         // generate global dict from dimension file
         if (carbonLoadModel.getDimFolderPath != null) {
@@ -812,7 +818,7 @@ object GlobalDictionaryUtil extends Logging {
       }
     } catch {
       case ex: Exception =>
-        logError("generate global dictionary failed")
+        logError("generate global dictionary failed", ex)
         throw ex
     }
   }


[40/50] [abbrv] incubator-carbondata git commit: handled all dictionary exception

Posted by ra...@apache.org.
handled all dictionary exception

add testcase

remove rdd.count

use better sentence


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

Branch: refs/heads/branch-0.1
Commit: 50b3746b3e0a8e74499689d520a0e77e433234ca
Parents: 30548ba
Author: foryou2030 <fo...@126.com>
Authored: Fri Sep 2 17:18:37 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:33:21 2016 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  4 --
 .../spark/util/GlobalDictionaryUtil.scala       | 73 +++++++++++++++-----
 integration/spark/src/test/resources/dict.txt   |  1 +
 .../dataload/TestLoadDataUseAllDictionary.scala | 56 +++++++++++++++
 4 files changed, 111 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/50b3746b/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 6050719..41d6ebf 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
@@ -199,10 +199,6 @@ public final class CarbonCommonConstants {
    */
   public static final String MEMBER_DEFAULT_VAL = "@NU#LL$!";
   /**
-   * DEFAULT_COLUMN_NAME
-   */
-  public static final String DEFAULT_COLUMN_NAME = "@NU#LL$!COLUMN";
-  /**
    * FILE STATUS IN-PROGRESS
    */
   public static final String FILE_INPROGRESS_STATUS = ".inprogress";

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/50b3746b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index 022a5f8..02b70d0 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -27,7 +27,7 @@ import scala.language.implicitConversions
 import scala.util.control.Breaks.{break, breakable}
 
 import org.apache.commons.lang3.{ArrayUtils, StringUtils}
-import org.apache.spark.Logging
+import org.apache.spark.{Accumulator, Logging}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.{CarbonEnv, CarbonRelation, DataFrame, SQLContext}
 import org.apache.spark.sql.hive.CarbonMetastoreCatalog
@@ -584,6 +584,48 @@ object GlobalDictionaryUtil extends Logging {
   }
 
   /**
+   * parse records in dictionary file and validate record
+   *
+   * @param x
+   * @param accum
+   * @param csvFileColumns
+   */
+  private def parseRecord(x: String, accum: Accumulator[Int],
+                  csvFileColumns: Array[String]) : (String, String) = {
+    val tokens = x.split("" + CSVWriter.DEFAULT_SEPARATOR)
+    var columnName: String = ""
+    var value: String = ""
+    // such as "," , "", throw ex
+    if (tokens.size == 0) {
+      logError("Read a bad dictionary record: " + x)
+      accum += 1
+    } else if (tokens.size == 1) {
+      // such as "1", "jone", throw ex
+      if (x.contains(",") == false) {
+        accum += 1
+      } else {
+        try {
+          columnName = csvFileColumns(tokens(0).toInt)
+        } catch {
+          case ex: Exception =>
+            logError("Read a bad dictionary record: " + x)
+            accum += 1
+        }
+      }
+    } else {
+      try {
+        columnName = csvFileColumns(tokens(0).toInt)
+        value = tokens(1)
+      } catch {
+        case ex: Exception =>
+          logError("Read a bad dictionary record: " + x)
+          accum += 1
+      }
+    }
+    (columnName, value)
+  }
+
+  /**
    * read local dictionary and prune column
    *
    * @param sqlContext
@@ -595,27 +637,13 @@ object GlobalDictionaryUtil extends Logging {
   private def readAllDictionaryFiles(sqlContext: SQLContext,
                                      csvFileColumns: Array[String],
                                      requireColumns: Array[String],
-                                     allDictionaryPath: String) = {
+                                     allDictionaryPath: String,
+                                     accumulator: Accumulator[Int]) = {
     var allDictionaryRdd: RDD[(String, Iterable[String])] = null
     try {
       // read local dictionary file, and spilt (columnIndex, columnValue)
       val basicRdd = sqlContext.sparkContext.textFile(allDictionaryPath)
-        .map(x => {
-        val tokens = x.split("" + CSVWriter.DEFAULT_SEPARATOR)
-        if (tokens.size != 2) {
-          logError("Read a bad dictionary record: " + x)
-        }
-        var columnName: String = CarbonCommonConstants.DEFAULT_COLUMN_NAME
-        var value: String = ""
-        try {
-          columnName = csvFileColumns(tokens(0).toInt)
-          value = tokens(1)
-        } catch {
-          case ex: Exception =>
-            logError("Reset bad dictionary record as default value")
-        }
-        (columnName, value)
-      })
+        .map(x => parseRecord(x, accumulator, csvFileColumns)).persist()
 
       // group by column index, and filter required columns
       val requireColumnsList = requireColumns.toList
@@ -801,9 +829,11 @@ object GlobalDictionaryUtil extends Logging {
           if (requireDimension.nonEmpty) {
             val model = createDictionaryLoadModel(carbonLoadModel, table, requireDimension,
               hdfsLocation, dictfolderPath, false)
+            // check if dictionary files contains bad record
+            val accumulator = sqlContext.sparkContext.accumulator(0)
             // read local dictionary file, and group by key
             val allDictionaryRdd = readAllDictionaryFiles(sqlContext, headers,
-              requireColumnNames, allDictionaryPath)
+              requireColumnNames, allDictionaryPath, accumulator)
             // read exist dictionary and combine
             val inputRDD = new CarbonAllDictionaryCombineRDD(allDictionaryRdd, model)
               .partitionBy(new ColumnPartitioner(model.primDimensions.length))
@@ -811,6 +841,11 @@ object GlobalDictionaryUtil extends Logging {
             val statusList = new CarbonGlobalDictionaryGenerateRDD(inputRDD, model).collect()
             // check result status
             checkStatus(carbonLoadModel, sqlContext, model, statusList)
+            // if the dictionary contains wrong format record, throw ex
+            if (accumulator.value > 0) {
+              throw new DataLoadingException("Data Loading failure, dictionary values are " +
+                "not in correct format!")
+            }
           } else {
             logInfo("have no column need to generate global dictionary")
           }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/50b3746b/integration/spark/src/test/resources/dict.txt
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/dict.txt b/integration/spark/src/test/resources/dict.txt
new file mode 100644
index 0000000..64ba47f
--- /dev/null
+++ b/integration/spark/src/test/resources/dict.txt
@@ -0,0 +1 @@
+12154
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/50b3746b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
new file mode 100644
index 0000000..143b386
--- /dev/null
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/dataload/TestLoadDataUseAllDictionary.scala
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.spark.testsuite.dataload
+
+import org.apache.carbondata.processing.etl.DataLoadingException
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestLoadDataUseAllDictionary extends QueryTest with BeforeAndAfterAll{
+  override def beforeAll {
+    sql("DROP TABLE IF EXISTS t3")
+    sql("""
+           CREATE TABLE IF NOT EXISTS t3
+           (ID Int, date Timestamp, country String,
+           name String, phonetype String, serialname String, salary Int)
+           STORED BY 'carbondata'
+           """)
+  }
+
+  test("test load data use all dictionary, and given wrong format dictionary values") {
+    try {
+      sql(s"""
+           LOAD DATA LOCAL INPATH './src/test/resources/windows.csv' into table t3
+           options('FILEHEADER'='id,date,country,name,phonetype,serialname,salary',
+           'All_DICTIONARY_PATH'='./src/test/resources/dict.txt')
+           """)
+      assert(false)
+    } catch {
+      case e: DataLoadingException =>
+        assert(e.getMessage.equals("Data Loading failure, dictionary values are " +
+          "not in correct format!"))
+    }
+  }
+
+  override def afterAll {
+    sql("DROP TABLE IF EXISTS t3")
+  }
+}


[45/50] [abbrv] incubator-carbondata git commit: Corrected file headers and removed unnecessary open csv files

Posted by ra...@apache.org.
Corrected file headers and removed unnecessary open csv files


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

Branch: refs/heads/branch-0.1
Commit: be1675f4948f02eb362830657bea67a5143bd154
Parents: 17b00c2
Author: ravipesala <ra...@gmail.com>
Authored: Wed Sep 21 21:30:55 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:35:19 2016 +0530

----------------------------------------------------------------------
 .../carbondata/scan/QueryExecutor_UT.java       |  35 --
 .../spark/partition/api/DataPartitioner.java    |   9 -
 .../spark/partition/api/Partition.java          |   9 -
 .../partition/api/impl/CSVFilePartitioner.java  | 365 ------------
 .../api/impl/DataPartitionerProperties.java     |   3 -
 .../partition/api/impl/DefaultLoadBalancer.java |   9 -
 .../spark/partition/api/impl/PartitionImpl.java |   9 -
 .../api/impl/QueryPartitionHelper.java          |   9 -
 .../spark/partition/reader/CSVIterator.java     |  74 ---
 .../spark/partition/reader/CSVParser.java       | 559 -------------------
 .../spark/partition/reader/CSVReader.java       | 496 ----------------
 .../spark/partition/reader/CSVWriter.java       | 396 -------------
 .../spark/partition/reader/LineReader.java      |  68 ---
 .../spark/partition/reader/ResultSetHelper.java |  87 ---
 .../reader/ResultSetHelperService.java          | 327 -----------
 .../spark/rdd/CarbonDataPartitionRDD.scala      | 112 ----
 .../spark/rdd/CarbonDataRDDFactory.scala        |  31 -
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |   7 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  23 +-
 .../org/apache/spark/sql/CarbonContext.scala    |  42 +-
 .../execution/command/carbonTableSchema.scala   |  50 --
 21 files changed, 20 insertions(+), 2700 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/core/src/test/java/org/apache/carbondata/scan/QueryExecutor_UT.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/scan/QueryExecutor_UT.java b/core/src/test/java/org/apache/carbondata/scan/QueryExecutor_UT.java
deleted file mode 100644
index 88fa999..0000000
--- a/core/src/test/java/org/apache/carbondata/scan/QueryExecutor_UT.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/**
- * Copyright Notice
- * =====================================
- * This file contains proprietary information of
- * Huawei Technologies India Pvt Ltd.
- * Copying or reproduction without prior written approval is prohibited.
- * Copyright (c) 2012
- * =====================================
- */
-package org.apache.carbondata.scan;
-
-import junit.framework.TestCase;
-
-public class QueryExecutor_UT extends TestCase {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.java
index 603e01e..58f3a2d 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/DataPartitioner.java
@@ -17,15 +17,6 @@
  * under the License.
  */
 
-/**
- * Copyright Notice
- * =====================================
- * This file contains proprietary information of
- * Huawei Technologies India Pvt Ltd.
- * Copying or reproduction without prior written approval is prohibited.
- * Copyright (c) 1997
- * =====================================
- */
 package org.apache.carbondata.spark.partition.api;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/Partition.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/Partition.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/Partition.java
index 5d94fa0..61639d3 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/Partition.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/Partition.java
@@ -17,15 +17,6 @@
  * under the License.
  */
 
-/**
- * Copyright Notice
- * =====================================
- * This file contains proprietary information of
- * Huawei Technologies India Pvt Ltd.
- * Copying or reproduction without prior written approval is prohibited.
- * Copyright (c) 1997
- * =====================================
- */
 package org.apache.carbondata.spark.partition.api;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/CSVFilePartitioner.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/CSVFilePartitioner.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/CSVFilePartitioner.java
deleted file mode 100644
index c61aafa..0000000
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/CSVFilePartitioner.java
+++ /dev/null
@@ -1,365 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/**
- * Copyright Notice
- * =====================================
- * This file contains proprietary information of
- * Huawei Technologies India Pvt Ltd.
- * Copying or reproduction without prior written approval is prohibited.
- * Copyright (c) 1997
- * =====================================
- */
-package org.apache.carbondata.spark.partition.api.impl;
-
-import java.io.DataInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.zip.GZIPInputStream;
-
-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.datastorage.store.filesystem.CarbonFile;
-import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
-import org.apache.carbondata.core.datastorage.store.impl.FileFactory.FileType;
-import org.apache.carbondata.core.util.CarbonProperties;
-import org.apache.carbondata.core.util.CarbonUtil;
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordslogger;
-import org.apache.carbondata.spark.partition.api.DataPartitioner;
-import org.apache.carbondata.spark.partition.api.Partition;
-import org.apache.carbondata.spark.partition.reader.CSVParser;
-import org.apache.carbondata.spark.partition.reader.CSVReader;
-import org.apache.carbondata.spark.partition.reader.CSVWriter;
-
-import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
-import org.apache.commons.lang.StringUtils;
-import org.apache.spark.sql.execution.command.Partitioner;
-
-/**
- * Split the CSV file into the number of partitions using the given partition information
- */
-public class CSVFilePartitioner {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CSVFilePartitioner.class.getName());
-  private String partitionerClass;
-  private String sourceFilesBasePath;
-  private boolean partialSuccess;
-  /**
-   * badRecordslogger
-   */
-  private BadRecordslogger badRecordslogger;
-
-  /**
-   * @param partitionerClass
-   */
-  public CSVFilePartitioner(String partitionerClass, String sourceFilesBasePath) {
-    this.partitionerClass = partitionerClass;
-    this.sourceFilesBasePath = sourceFilesBasePath;
-  }
-
-  public boolean isPartialSuccess() {
-    return partialSuccess;
-  }
-
-  @Deprecated public void splitFile(String databaseName, String tableName,
-      List<String> sourceFilePath, String targetFolder, List<String> nodes, int partitionCount,
-      String[] partitionColumn, String[] requiredColumns, String delimiter, String quoteChar,
-      String fileHeader, String escapeChar, boolean multiLine) throws Exception {
-    LOGGER
-        .info("Processing file split: " + sourceFilePath);
-
-    // Create the target folder
-    FileFactory.mkdirs(targetFolder, FileFactory.getFileType(targetFolder));
-
-    String[] headerColumns = null;
-
-    HashMap<Partition, CSVWriter> outputStreamsMap =
-        new HashMap<Partition, CSVWriter>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    String key = databaseName + '_' + tableName;
-    badRecordslogger = new BadRecordslogger(key, "Partition_" + System.currentTimeMillis() + ".log",
-        getBadLogStoreLocation("partition/" + databaseName + '/' + tableName));
-
-    CSVReader dataInputStream = null;
-
-    long recordCounter = 0;
-
-    CSVParser customParser = getCustomParser(delimiter, quoteChar, escapeChar);
-
-    for (int i = 0; i < sourceFilePath.size(); i++) {
-      try {
-        CarbonFile file = FileFactory
-            .getCarbonFile(sourceFilePath.get(i), FileFactory.getFileType(sourceFilePath.get(i)));
-        // File file = new File(sourceFilePath);
-        String fileAbsolutePath = file.getAbsolutePath();
-        String fileName = null;
-        if (!sourceFilesBasePath.endsWith(".csv") && fileAbsolutePath
-            .startsWith(sourceFilesBasePath)) {
-          if (sourceFilesBasePath.endsWith(File.separator)) {
-            fileName = fileAbsolutePath.substring(sourceFilesBasePath.length())
-                .replace(File.separator, "_");
-          } else {
-            fileName = fileAbsolutePath.substring(sourceFilesBasePath.length() + 1)
-                .replace(File.separator, "_");
-          }
-        } else {
-          fileName = file.getName();
-        }
-
-        // Read and prepare columns from first row in file
-        DataInputStream inputStream = FileFactory.getDataInputStream(sourceFilePath.get(i),
-            FileFactory.getFileType(sourceFilePath.get(i)));
-        if (fileName.endsWith(".gz")) {
-          GZIPInputStream gzipInputStream = new GZIPInputStream(inputStream);
-          dataInputStream =
-              new CSVReader(new InputStreamReader(gzipInputStream, Charset.defaultCharset()),
-                  CSVReader.DEFAULT_SKIP_LINES, customParser);
-          fileName = fileName.substring(0, fileName.indexOf(".gz"));
-        } else if (fileName.endsWith(".bz2")) {
-          BZip2CompressorInputStream stream = new BZip2CompressorInputStream(inputStream);
-          dataInputStream = new CSVReader(new InputStreamReader(stream, Charset.defaultCharset()),
-              CSVReader.DEFAULT_SKIP_LINES, customParser);
-          fileName = fileName.substring(0, fileName.indexOf(".bz2"));
-        } else if (fileName.endsWith(".csv")) {
-          dataInputStream =
-              new CSVReader(new InputStreamReader(inputStream, Charset.defaultCharset()),
-                  CSVReader.DEFAULT_SKIP_LINES, customParser);
-          fileName = fileName.substring(0, fileName.indexOf(".csv"));
-        } else {
-          LOGGER.info("Processing file split: Unsupported File Extension: Skipping File : "
-              + file.getAbsolutePath());
-          partialSuccess = true;
-          return;
-        }
-        dataInputStream.setBadRecordsLogger(badRecordslogger);
-        if (fileHeader == null || fileHeader.length() == 0) {
-          headerColumns = dataInputStream.readNext();
-        } else {
-          headerColumns = fileHeader.split(",");
-        }
-        if (null == headerColumns) {
-          LOGGER.info("Csv file does not contain the header column neither the headers are "
-                  + "passed in DDL or API. Skipping file :: " + sourceFilePath);
-          partialSuccess = true;
-          return;
-        }
-        int[] indexes = pruneColumnsAndGetIndexes(headerColumns, requiredColumns);
-
-        // In case there is a dummy measure required columns length and
-        // header columns length will not be equal
-        if ((null == fileHeader || 0 == fileHeader.length()) && (0 == indexes.length) && (
-            fileHeader.length() != indexes.length)) {
-          LOGGER.info("Column headers are invalid. They do not match with the schema headers."
-                  + "Skipping file :: " + sourceFilePath);
-          partialSuccess = true;
-          return;
-        }
-
-        partitionData(targetFolder, nodes, partitionCount, partitionColumn, headerColumns,
-            outputStreamsMap, dataInputStream, recordCounter, fileName, indexes, fileAbsolutePath);
-      } catch (IOException e) {
-        LOGGER.error(e, e.getMessage());
-      } finally {
-        CarbonUtil.closeStreams(dataInputStream);
-
-        for (CSVWriter dataOutStream : outputStreamsMap.values()) {
-          CarbonUtil.closeStreams(dataOutStream);
-        }
-        badRecordslogger.closeStreams();
-      }
-    }
-  }
-
-  private void partitionData(String targetFolder, List<String> nodes, int partitionCount,
-      String[] partitionColumn, String[] headerColumns,
-      HashMap<Partition, CSVWriter> outputStreamsMap, CSVReader dataInputStream, long recordCounter,
-      String fileName, int[] indexes, String fileAbsolutePath)
-      throws InstantiationException, IllegalAccessException, ClassNotFoundException, IOException {
-    DataPartitioner dataPartitioner =
-        getDataPartitioner(targetFolder, nodes, partitionCount, partitionColumn, headerColumns);
-
-    //Get partitions and create output streams
-    List<Partition> allPartitions = dataPartitioner.getAllPartitions();
-
-    loopPartitionsAndPopulateOutStreamMap(outputStreamsMap, fileName, allPartitions);
-
-    //Write header in all the target files
-    for (CSVWriter dataOutStream : outputStreamsMap.values()) {
-      dataOutStream.writeNext(pruneColumns(headerColumns, indexes));
-    }
-
-    recordCounter = writeTargetStream(outputStreamsMap, dataInputStream, recordCounter, indexes,
-        dataPartitioner, headerColumns, fileAbsolutePath);
-
-    LOGGER
-        .info("Processed Record count: " + recordCounter);
-  }
-
-  private CSVParser getCustomParser(String delimiter, String quoteChar, String escapeChar) {
-    CSVParser customParser = null;
-    boolean ignoreQuote = false;
-    boolean ignoreEscape = false;
-    char defaultQuoteChar = CSVParser.DEFAULT_QUOTE_CHARACTER;
-    char defaultEscapeChar = CSVParser.DEFAULT_ESCAPE_CHARACTER;
-    if (quoteChar == null || quoteChar.isEmpty() || quoteChar.trim().isEmpty()) {
-      ignoreQuote = true;
-    } else {
-      ignoreQuote = false;
-      defaultQuoteChar = quoteChar.charAt(0);
-    }
-    if (escapeChar == null || escapeChar.isEmpty() || escapeChar.trim().isEmpty()) {
-      ignoreEscape = true;
-    } else {
-      ignoreEscape = false;
-      defaultEscapeChar = escapeChar.charAt(0);
-    }
-    delimiter = CarbonUtil.unescapeChar(delimiter);
-    customParser = new CSVParser(delimiter.charAt(0), defaultQuoteChar, defaultEscapeChar,
-        CSVParser.DEFAULT_STRICT_QUOTES, CSVParser.DEFAULT_IGNORE_LEADING_WHITESPACE, ignoreQuote,
-        ignoreEscape);
-    return customParser;
-  }
-
-  private DataPartitioner getDataPartitioner(String targetFolder, List<String> nodes,
-      int partitionCount, String[] partitionColumn, String[] headerColumns)
-      throws InstantiationException, IllegalAccessException, ClassNotFoundException {
-    DataPartitioner dataPartitioner =
-        (DataPartitioner) Class.forName(partitionerClass).newInstance();
-
-    Partitioner partitioner = new Partitioner(partitionerClass, partitionColumn, partitionCount,
-        nodes.toArray(new String[nodes.size()]));
-    //Initialise the partitioner
-    dataPartitioner.initialize(targetFolder, headerColumns, partitioner);
-    return dataPartitioner;
-  }
-
-  private long writeTargetStream(HashMap<Partition, CSVWriter> outputStreamsMap,
-      CSVReader dataInputStream, long recordCounter, int[] indexes, DataPartitioner dataPartitioner,
-      String[] headerColumns, String fileAbsolutePath) throws IOException {
-    String[] record = null;
-    Partition tartgetPartition = null;
-    CSVWriter targetStream = null;
-    record = dataInputStream.readNext();
-    int skippedLines = 0;
-    if (null == record) {
-      return recordCounter;
-    } else {
-      boolean isEqual = compareHeaderColumnWithFirstRecordInCSV(headerColumns, record);
-      if (isEqual) {
-        record = dataInputStream.readNext();
-        recordCounter++;
-      }
-    }
-    while (null != record) {
-      tartgetPartition = dataPartitioner.getPartionForTuple(record, recordCounter);
-      targetStream = outputStreamsMap.get(tartgetPartition);
-      try {
-        targetStream.writeNext(pruneColumns(record, indexes));
-      } catch (ArrayIndexOutOfBoundsException e) {
-        partialSuccess = true;
-        skippedLines++;
-        badRecordslogger.addBadRecordsToBilder(record, record.length,
-            "No. of columns not matched with table columns", null);
-        LOGGER.error("BAD Record Found: No. of columns not matched with table columns, "
-            + "Skipping line: (" + (recordCounter + 1) + ") in File :" + fileAbsolutePath);
-      } catch (Exception e) {
-        partialSuccess = true;
-        skippedLines++;
-        badRecordslogger.addBadRecordsToBilder(record, record.length, e.getMessage(), null);
-        LOGGER.info("Exception while processing the record at line " + (recordCounter + 1)
-            + " in partiton " + tartgetPartition.getUniqueID());
-      } finally {
-        record = dataInputStream.readNext();
-        recordCounter++;
-      }
-    }
-    if (skippedLines != 0) {
-      LOGGER.info("No. of bad records skipped: (" + skippedLines + ") in file:" + fileAbsolutePath);
-    }
-    return recordCounter;
-  }
-
-  private boolean compareHeaderColumnWithFirstRecordInCSV(String[] headerColumns,
-      String[] firstRecord) {
-    String header = StringUtils.join(headerColumns, ',');
-    String record = StringUtils.join(firstRecord, ',');
-    if (header != null && header.equals(record)) {
-      return true;
-    }
-    return false;
-  }
-
-  private void loopPartitionsAndPopulateOutStreamMap(HashMap<Partition, CSVWriter> outputStreamsMap,
-      String fileName, List<Partition> allPartitions) throws IOException {
-    for (Partition partition : allPartitions) {
-      String targetFolderPath = partition.getFilePath();
-      FileType fileType = FileFactory.getFileType(targetFolderPath);
-      FileFactory.mkdirs(targetFolderPath, fileType);
-      outputStreamsMap.put(partition, new CSVWriter(new OutputStreamWriter(FileFactory
-          .getDataOutputStream(
-              targetFolderPath + '/' + fileName + '_' + partition.getUniqueID() + ".csv", fileType,
-              (short) 1), Charset.defaultCharset())));
-    }
-  }
-
-  private int[] pruneColumnsAndGetIndexes(String[] headerColumns, String[] requiredColumns) {
-    if (requiredColumns == null) {
-      requiredColumns = headerColumns;
-    }
-    List<Integer> indexesList = new ArrayList<Integer>();
-    for (int i = 0; i < headerColumns.length; i++) {
-      for (int j = 0; j < requiredColumns.length; j++) {
-        if (headerColumns[i].equalsIgnoreCase(requiredColumns[j])) {
-          indexesList.add(i);
-          break;
-        }
-      }
-    }
-    int[] indexes = new int[indexesList.size()];
-    for (int i = 0; i < indexesList.size(); i++) {
-      indexes[i] = indexesList.get(i);
-    }
-    return indexes;
-  }
-
-  private String[] pruneColumns(String[] tuple, int[] indexes) {
-    String[] sb = new String[indexes.length];
-    int length = indexes.length;
-    for (int i = 0; i < length; i++) {
-      sb[i] = tuple[indexes[i]];
-    }
-    return sb;
-  }
-
-  private String getBadLogStoreLocation(String storeLocation) {
-    String badLogStoreLocation =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC);
-    badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation;
-
-    return badLogStoreLocation;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DataPartitionerProperties.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DataPartitionerProperties.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DataPartitionerProperties.java
index 8a80c4e..bc6e54f 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DataPartitionerProperties.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DataPartitionerProperties.java
@@ -17,9 +17,6 @@
  * under the License.
  */
 
-/**
- *
- */
 package org.apache.carbondata.spark.partition.api.impl;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java
index 67c257d..9bee8a2 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/DefaultLoadBalancer.java
@@ -17,15 +17,6 @@
  * under the License.
  */
 
-/**
- * Copyright Notice
- * =====================================
- * This file contains proprietary information of
- * Huawei Technologies India Pvt Ltd.
- * Copying or reproduction without prior written approval is prohibited.
- * Copyright (c) 1997
- * =====================================
- */
 package org.apache.carbondata.spark.partition.api.impl;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionImpl.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionImpl.java
index b49db59..bd7cc42 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionImpl.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/PartitionImpl.java
@@ -17,15 +17,6 @@
  * under the License.
  */
 
-/**
- * Copyright Notice
- * =====================================
- * This file contains proprietary information of
- * Huawei Technologies India Pvt Ltd.
- * Copying or reproduction without prior written approval is prohibited.
- * Copyright (c) 1997
- * =====================================
- */
 package org.apache.carbondata.spark.partition.api.impl;
 
 import java.util.List;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
index 37d772d..72e7b08 100644
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
+++ b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
@@ -17,15 +17,6 @@
  * under the License.
  */
 
-/**
- * Copyright Notice
- * =====================================
- * This file contains proprietary information of
- * Huawei Technologies India Pvt Ltd.
- * Copying or reproduction without prior written approval is prohibited.
- * Copyright (c) 1997
- * =====================================
- */
 package org.apache.carbondata.spark.partition.api.impl;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVIterator.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVIterator.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVIterator.java
deleted file mode 100644
index d57f708..0000000
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVIterator.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.carbondata.spark.partition.reader;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-/**
- * Provides an Iterator over the data found in opencsv.
- */
-public class CSVIterator implements Iterator<String[]> {
-  private CSVReader reader;
-  private String[] nextLine;
-
-  /**
-   * @param reader reader for the csv data.
-   * @throws IOException if unable to read data from the reader.
-   */
-  public CSVIterator(CSVReader reader) throws IOException {
-    this.reader = reader;
-    nextLine = reader.readNext();
-  }
-
-  /**
-   * Returns true if the iteration has more elements.
-   * In other words, returns true if next() would return an element rather
-   * than throwing an exception.
-   *
-   * @return true if the CSVIterator has more elements.
-   */
-  public boolean hasNext() {
-    return nextLine != null;
-  }
-
-  /**
-   * Returns the next elenebt in the iterator.
-   *
-   * @return The next element of the iterator.
-   */
-  public String[] next() {
-    String[] temp = nextLine;
-    try {
-      nextLine = reader.readNext();
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return temp;
-  }
-
-  /**
-   * This method is not supported by openCSV and will throw a UnsupportedOperationException
-   * if called.
-   */
-  public void remove() {
-    throw new UnsupportedOperationException("This is a read only iterator.");
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVParser.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVParser.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVParser.java
deleted file mode 100644
index a052f31..0000000
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVParser.java
+++ /dev/null
@@ -1,559 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.carbondata.spark.partition.reader;
-
-/**
- * Copyright 2005 Bytecode Pty Ltd.
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * A very simple CSV parser released under a commercial-friendly license.
- * This just implements splitting a single line into fields.
- *
- * @author Glen Smith
- * @author Rainer Pruy
- */
-public class CSVParser {
-
-  /**
-   * The default separator to use if none is supplied to the constructor.
-   */
-  public static final char DEFAULT_SEPARATOR = ',';
-  /**
-   * The average size of a line read by openCSV (used for setting the size of StringBuilders).
-   */
-  public static final int INITIAL_READ_SIZE = 128;
-  /**
-   * The default quote character to use if none is supplied to the
-   * constructor.
-   */
-  public static final char DEFAULT_QUOTE_CHARACTER = '"';
-  /**
-   * The default escape character to use if none is supplied to the
-   * constructor.
-   */
-  public static final char DEFAULT_ESCAPE_CHARACTER = '\\';
-  /**
-   * The default strict quote behavior to use if none is supplied to the
-   * constructor.
-   */
-  public static final boolean DEFAULT_STRICT_QUOTES = false;
-  /**
-   * The default leading whitespace behavior to use if none is supplied to the
-   * constructor.
-   */
-  public static final boolean DEFAULT_IGNORE_LEADING_WHITESPACE = true;
-  /**
-   * If the quote character is set to null then there is no quote character.
-   */
-  public static final boolean DEFAULT_IGNORE_QUOTATIONS = false;
-  /**
-   * This is the "null" character - if a value is set to this then it is ignored.
-   */
-  public static final char NULL_CHARACTER = '\0';
-  /**
-   * This is the character that the CSVParser will treat as the separator.
-   */
-  private final char separator;
-  /**
-   * This is the character that the CSVParser will treat as the quotation character.
-   */
-  private final char quotechar;
-  /**
-   * This is the character that the CSVParser will treat as the escape character.
-   */
-  private final char escape;
-  /**
-   * Determines if the field is between quotes (true) or between separators (false).
-   */
-  private final boolean strictQuotes;
-  /**
-   * Ignore any leading white space at the start of the field.
-   */
-  private final boolean ignoreLeadingWhiteSpace;
-  /**
-   * Skip over quotation characters when parsing.
-   */
-  private final boolean ignoreQuotations;
-  private boolean ignoreEscapeChar;
-  private String pending;
-  private boolean inField;
-  private int charCountInsideQuote;
-
-  /**
-   * Constructs CSVParser using a comma for the separator.
-   */
-  public CSVParser() {
-    this(DEFAULT_SEPARATOR, DEFAULT_QUOTE_CHARACTER, DEFAULT_ESCAPE_CHARACTER);
-  }
-
-  /**
-   * Constructs CSVParser with supplied separator.
-   *
-   * @param separator the delimiter to use for separating entries.
-   */
-  public CSVParser(char separator) {
-    this(separator, DEFAULT_QUOTE_CHARACTER, DEFAULT_ESCAPE_CHARACTER);
-  }
-
-  /**
-   * Constructs CSVParser with supplied separator and quote char.
-   *
-   * @param separator the delimiter to use for separating entries
-   * @param quotechar the character to use for quoted elements
-   */
-  public CSVParser(char separator, char quotechar) {
-    this(separator, quotechar, DEFAULT_ESCAPE_CHARACTER);
-  }
-
-  /**
-   * Constructs CSVReader with supplied separator and quote char.
-   *
-   * @param separator the delimiter to use for separating entries
-   * @param quotechar the character to use for quoted elements
-   * @param escape    the character to use for escaping a separator or quote
-   */
-  public CSVParser(char separator, char quotechar, char escape) {
-    this(separator, quotechar, escape, DEFAULT_STRICT_QUOTES);
-  }
-
-  /**
-   * Constructs CSVParser with supplied separator and quote char.
-   * Allows setting the "strict quotes" flag
-   *
-   * @param separator    the delimiter to use for separating entries
-   * @param quotechar    the character to use for quoted elements
-   * @param escape       the character to use for escaping a separator or quote
-   * @param strictQuotes if true, characters outside the quotes are ignored
-   */
-  public CSVParser(char separator, char quotechar, char escape, boolean strictQuotes) {
-    this(separator, quotechar, escape, strictQuotes, DEFAULT_IGNORE_LEADING_WHITESPACE);
-  }
-
-  /**
-   * Constructs CSVParser with supplied separator and quote char.
-   * Allows setting the "strict quotes" and "ignore leading whitespace" flags
-   *
-   * @param separator               the delimiter to use for separating entries
-   * @param quotechar               the character to use for quoted elements
-   * @param escape                  the character to use for escaping a separator or quote
-   * @param strictQuotes            if true, characters outside the quotes are ignored
-   * @param ignoreLeadingWhiteSpace if true, white space in front of a quote in a field is ignored
-   */
-  public CSVParser(char separator, char quotechar, char escape, boolean strictQuotes,
-      boolean ignoreLeadingWhiteSpace) {
-    this(separator, quotechar, escape, strictQuotes, ignoreLeadingWhiteSpace,
-        DEFAULT_IGNORE_QUOTATIONS);
-  }
-
-  /**
-   * Constructs CSVParser with supplied separator and quote char.
-   * Allows setting the "strict quotes" and "ignore leading whitespace" flags
-   *
-   * @param separator               the delimiter to use for separating entries
-   * @param quotechar               the character to use for quoted elements
-   * @param escape                  the character to use for escaping a separator or quote
-   * @param strictQuotes            if true, characters outside the quotes are ignored
-   * @param ignoreLeadingWhiteSpace if true, white space in front of a quote in a field is ignored
-   * @param ignoreQuotations        if true, treat quotations like any other character.
-   */
-  public CSVParser(char separator, char quotechar, char escape, boolean strictQuotes,
-      boolean ignoreLeadingWhiteSpace, boolean ignoreQuotations) {
-    if (anyCharactersAreTheSame(separator, quotechar, escape)) {
-      throw new UnsupportedOperationException(
-          "The separator, quote, and escape characters must be different!");
-    }
-    if (separator == NULL_CHARACTER) {
-      throw new UnsupportedOperationException("The separator character must be defined!");
-    }
-    this.separator = separator;
-    this.quotechar = quotechar;
-    this.escape = escape;
-    this.strictQuotes = strictQuotes;
-    this.ignoreLeadingWhiteSpace = ignoreLeadingWhiteSpace;
-    this.ignoreQuotations = ignoreQuotations;
-  }
-
-  /**
-   * Constructs CSVParser with supplied separator and quote char.
-   * Allows setting the "strict quotes" and "ignore leading whitespace" flags
-   *
-   * @param separator               the delimiter to use for separating entries
-   * @param quotechar               the character to use for quoted elements
-   * @param escape                  the character to use for escaping a separator or quote
-   * @param strictQuotes            if true, characters outside the quotes are ignored
-   * @param ignoreLeadingWhiteSpace if true, white space in front of a quote in a field is ignored
-   * @param ignoreQuotations        if true, treat quotations like any other character.
-   */
-  public CSVParser(char separator, char quotechar, char escape, boolean strictQuotes,
-      boolean ignoreLeadingWhiteSpace, boolean ignoreQuotations, boolean ignoreEscape) {
-    this(separator, quotechar, escape, strictQuotes, ignoreLeadingWhiteSpace, ignoreQuotations);
-    this.ignoreEscapeChar = ignoreEscape;
-  }
-
-  /**
-   * @return The default separator for this parser.
-   */
-  public char getSeparator() {
-    return separator;
-  }
-
-  /**
-   * @return The default quotation character for this parser.
-   */
-  public char getQuotechar() {
-    return quotechar;
-  }
-
-  /**
-   * @return The default escape character for this parser.
-   */
-  public char getEscape() {
-    return escape;
-  }
-
-  /**
-   * @return The default strictQuotes setting for this parser.
-   */
-  public boolean isStrictQuotes() {
-    return strictQuotes;
-  }
-
-  /**
-   * @return The default ignoreLeadingWhiteSpace setting for this parser.
-   */
-  public boolean isIgnoreLeadingWhiteSpace() {
-    return ignoreLeadingWhiteSpace;
-  }
-
-  /**
-   * @return the default ignoreQuotation setting for this parser.
-   */
-  public boolean isIgnoreQuotations() {
-    return ignoreQuotations;
-  }
-
-  /**
-   * checks to see if any two of the three characters are the same.  This is because in openCSV
-   * the separator, quote, and escape characters must the different.
-   *
-   * @param separator the defined separator character
-   * @param quotechar the defined quotation cahracter
-   * @param escape    the defined escape character
-   * @return true if any two of the three are the same.
-   */
-  private boolean anyCharactersAreTheSame(char separator, char quotechar, char escape) {
-    return isSameCharacter(separator, quotechar) || isSameCharacter(separator, escape)
-        || isSameCharacter(quotechar, escape);
-  }
-
-  /**
-   * checks that the two characters are the same and are not the defined NULL_CHARACTER.
-   *
-   * @param c1 first character
-   * @param c2 second character
-   * @return true if both characters are the same and are not the defined NULL_CHARACTER
-   */
-  private boolean isSameCharacter(char c1, char c2) {
-    return c1 != NULL_CHARACTER && c1 == c2;
-  }
-
-  /**
-   * @return true if something was left over from last call(s)
-   */
-  public boolean isPending() {
-    return pending != null;
-  }
-
-  public void setPending(String pending) {
-    this.pending = pending;
-  }
-
-  /**
-   * Parses an incoming String and returns an array of elements.  This method is used when the
-   * data spans multiple lines.
-   *
-   * @param nextLine current line to be processed
-   * @return the comma-tokenized list of elements, or null if nextLine is null
-   * @throws IOException if bad things happen during the read
-   */
-  public String[] parseLineMulti(String nextLine) throws IOException {
-    return parseLine(nextLine, true);
-  }
-
-  /**
-   * Parses an incoming String and returns an array of elements.  This method is used when all
-   * data is contained in a single line.
-   *
-   * @param nextLine Line to be parsed.
-   * @return the comma-tokenized list of elements, or null if nextLine is null
-   * @throws IOException if bad things happen during the read
-   */
-  public String[] parseLine(String nextLine) throws IOException {
-    return parseLine(nextLine, false);
-  }
-
-  /**
-   * Parses an incoming String and returns an array of elements.
-   *
-   * @param nextLine the string to parse
-   * @param multi    Does it take multiple lines to form a single record.
-   * @return the comma-tokenized list of elements, or null if nextLine is null
-   * @throws IOException if bad things happen during the read
-   */
-  private String[] parseLine(String nextLine, boolean multi) throws IOException {
-
-    if (!multi && pending != null) {
-      pending = null;
-    }
-
-    if (nextLine == null) {
-      if (pending != null) {
-        String s = pending;
-        pending = null;
-        return new String[] { s };
-      } else {
-        return null;
-      }
-    }
-
-    List<String> tokensOnThisLine = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    //CHECKSTYLE:OFF    Approval No:Approval-V1R2C10_010
-    StringBuilder sb = new StringBuilder(INITIAL_READ_SIZE);
-
-    boolean inQuotes = false;
-    // CHECKSTYLE:ON
-    if (pending != null) {
-      sb.append(pending);
-      pending = null;
-      inQuotes = !this.ignoreQuotations;//true;
-    }
-    inQuotes = checkForQuotes(nextLine, tokensOnThisLine, sb, inQuotes);
-    // line is done - check status
-    if ((inQuotes && !ignoreQuotations)) {
-      if (multi) {
-        // continuing a quoted section, re-append newline
-        sb.append('\n');
-        charCountInsideQuote = sb.length() - (sb.indexOf("\"") + 1);
-        if (charCountInsideQuote >= 10000) //max char count to wait till the quote terminates
-        {
-          throw new IOException("Un-terminated quoted field after 10000 characters");
-        }
-        pending = sb.toString();
-        sb = null; // this partial content is not to be added to field list yet
-      } else {
-        throw new IOException("Un-terminated quoted field at end of CSV line");
-      }
-    } else {
-      inField = false;
-      charCountInsideQuote = 0;
-    }
-
-    if (sb != null) {
-      tokensOnThisLine.add(sb.toString());
-    }
-    return tokensOnThisLine.toArray(new String[tokensOnThisLine.size()]);
-
-  }
-
-  /**
-   * @param nextLine
-   * @param tokensOnThisLine
-   * @param sb
-   * @param inQuotes
-   * @return
-   */
-  private boolean checkForQuotes(String nextLine, List<String> tokensOnThisLine, StringBuilder sb,
-      boolean inQuotes) {
-    for (int i = 0; i < nextLine.length(); i++) {
-
-      char c = nextLine.charAt(i);
-      //Naresh 00902756 : Modified to skip escape character check
-      if (!this.ignoreEscapeChar && c == this.escape) {
-        if (isNextCharacterEscapable(nextLine, inQuotes(inQuotes), i)) {
-          i = appendNextCharacterAndAdvanceLoop(nextLine, sb, i);
-        }
-      } else if (c == quotechar) {
-        if (isNextCharacterEscapedQuote(nextLine, inQuotes(inQuotes), i)) {
-          i = appendNextCharacterAndAdvanceLoop(nextLine, sb, i);
-        } else {
-          inQuotes = checkForStrictQuotes(nextLine, sb, inQuotes, i, c);
-        }
-        inField = !inField;
-      } else if (c == separator && !(inQuotes && !ignoreQuotations)) {
-        tokensOnThisLine.add(sb.toString());
-        sb.setLength(0);
-        inField = false;
-      } else {
-        if (!strictQuotes || (inQuotes && !ignoreQuotations)) {
-          sb.append(c);
-          inField = true;
-        }
-      }
-    }
-    return inQuotes;
-  }
-
-  /**
-   * @param nextLine
-   * @param sb
-   * @param inQuotes
-   * @param i
-   * @param c
-   * @return
-   */
-  private boolean checkForStrictQuotes(String nextLine, StringBuilder sb, boolean inQuotes, int i,
-      char c) {
-    inQuotes = !inQuotes;
-
-    // the tricky case of an embedded quote in the middle: a,bc"d"ef,g
-    if (!strictQuotes) {
-      if (i > 2 //not on the beginning of the line
-          && nextLine.charAt(i - 1) != this.separator
-          //not at the beginning of an escape sequence
-          && nextLine.length() > (i + 1) &&
-          //not at the end of an escape sequence
-          nextLine.charAt(i + 1) != this.separator) {
-        if (ignoreLeadingWhiteSpace && sb.length() > 0 && isAllWhiteSpace(sb)) {
-          sb.setLength(0);
-        } else {
-          sb.append(c);
-        }
-
-      }
-    }
-    return inQuotes;
-  }
-
-  /**
-   * Appends the next character in the line to the stringbuffer.
-   *
-   * @param line - line to process
-   * @param sb   - contains the processed character
-   * @param i    - current position in the line.
-   * @return new position in the line.
-   */
-  private int appendNextCharacterAndAdvanceLoop(String line, StringBuilder sb, int i) {
-    sb.append(line.charAt(i + 1));
-    i++;
-    return i;
-  }
-
-  /**
-   * Determines if we can process as if we were in quotes.
-   *
-   * @param inQuotes - are we currently in quotes.
-   * @return - true if we should process as if we are inside quotes.
-   */
-  private boolean inQuotes(boolean inQuotes) {
-    return (inQuotes && !ignoreQuotations) || inField;
-  }
-
-  /**
-   * Checks to see if the character after the index is a quotation character.
-   * precondition: the current character is a quote or an escape
-   *
-   * @param nextLine the current line
-   * @param inQuotes true if the current context is quoted
-   * @param i        current index in line
-   * @return true if the following character is a quote
-   */
-  private boolean isNextCharacterEscapedQuote(String nextLine, boolean inQuotes, int i) {
-    return inQuotes  // we are in quotes, therefore there can be escaped quotes in here.
-        && nextLine.length() > (i + 1)  // there is indeed another character to check.
-        && isCharacterQuoteCharacter(nextLine.charAt(i + 1));
-  }
-
-  /**
-   * Checks to see if the passed in character is the defined quotation character.
-   *
-   * @param c source character
-   * @return true if c is the defined quotation character
-   */
-  private boolean isCharacterQuoteCharacter(char c) {
-    return c == quotechar;
-  }
-
-  /**
-   * checks to see if the character is the defined escape character.
-   *
-   * @param c source character
-   * @return true if the character is the defined escape character
-   */
-  private boolean isCharacterEscapeCharacter(char c) {
-    return c == escape;
-  }
-
-  /**
-   * Checks to see if the character passed in could be escapable.  Escapable characters for
-   * openCSV are the quotation character or the escape character.
-   *
-   * @param c source character
-   * @return true if the character could be escapable.
-   */
-  private boolean isCharacterEscapable(char c) {
-    return isCharacterQuoteCharacter(c) || isCharacterEscapeCharacter(c);
-  }
-
-  /**
-   * Checks to see if the character after the current index in a String is an escapable character.
-   * Meaning the next character is either a quotation character or the escape char and you are
-   * inside quotes.
-   * precondition: the current character is an escape
-   *
-   * @param nextLine the current line
-   * @param inQuotes true if the current context is quoted
-   * @param i        current index in line
-   * @return true if the following character is a quote
-   */
-  protected boolean isNextCharacterEscapable(String nextLine, boolean inQuotes, int i) {
-    return inQuotes  // we are in quotes, therefore there can be escaped quotes in here.
-        && nextLine.length() > (i + 1)  // there is indeed another character to check.
-        && isCharacterEscapable(nextLine.charAt(i + 1));
-  }
-
-  /**
-   * Checks if every element is the character sequence is whitespace.
-   * precondition: sb.length() is greater than 0
-   *
-   * @param sb A sequence of characters to examine
-   * @return true if every character in the sequence is whitespace
-   */
-  protected boolean isAllWhiteSpace(CharSequence sb) {
-    return StringUtils.isWhitespace(sb);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVReader.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVReader.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVReader.java
deleted file mode 100644
index ce3f1b4..0000000
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVReader.java
+++ /dev/null
@@ -1,496 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.carbondata.spark.partition.reader;
-
-/**
- * Copyright 2005 Bytecode Pty Ltd.
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.BufferedReader;
-import java.io.Closeable;
-import java.io.IOException;
-import java.io.Reader;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-import org.apache.carbondata.core.constants.CarbonCommonConstants;
-import org.apache.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordslogger;
-
-/**
- * A very simple CSV reader released under a commercial-friendly license.
- *
- * @author Glen Smith
- */
-public class CSVReader implements Closeable, Iterable<String[]> {
-
-  public static final boolean DEFAULT_KEEP_CR = false;
-  public static final boolean DEFAULT_VERIFY_READER = true;
-  /**
-   * The default line to start reading.
-   */
-  public static final int DEFAULT_SKIP_LINES = 0;
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CSVReader.class.getName());
-  private CSVParser parser;
-  private int skipLines;
-  private BufferedReader br;
-  private LineReader lineReader;
-  private boolean hasNext = true;
-  private boolean linesSkiped;
-  private boolean keepCR;
-  private boolean verifyReader;
-  private long lineNum;
-  private long skippedLines;
-  private boolean multiLine = true;
-  private BadRecordslogger badRecordslogger;
-
-  /**
-   * Constructs CSVReader using a comma for the separator.
-   *
-   * @param reader the reader to an underlying CSV source.
-   */
-  public CSVReader(Reader reader) {
-    this(reader, CSVParser.DEFAULT_SEPARATOR, CSVParser.DEFAULT_QUOTE_CHARACTER,
-        CSVParser.DEFAULT_ESCAPE_CHARACTER);
-  }
-
-  /**
-   * Constructs CSVReader with supplied separator.
-   *
-   * @param reader    the reader to an underlying CSV source.
-   * @param separator the delimiter to use for separating entries.
-   */
-  public CSVReader(Reader reader, char separator) {
-    this(reader, separator, CSVParser.DEFAULT_QUOTE_CHARACTER, CSVParser.DEFAULT_ESCAPE_CHARACTER);
-  }
-
-  /**
-   * Constructs CSVReader with supplied separator and quote char.
-   *
-   * @param reader    the reader to an underlying CSV source.
-   * @param separator the delimiter to use for separating entries
-   * @param quotechar the character to use for quoted elements
-   */
-  public CSVReader(Reader reader, char separator, char quotechar) {
-    this(reader, separator, quotechar, CSVParser.DEFAULT_ESCAPE_CHARACTER, DEFAULT_SKIP_LINES,
-        CSVParser.DEFAULT_STRICT_QUOTES);
-  }
-
-  /**
-   * Constructs CSVReader with supplied separator, quote char and quote handling
-   * behavior.
-   *
-   * @param reader       the reader to an underlying CSV source.
-   * @param separator    the delimiter to use for separating entries
-   * @param quotechar    the character to use for quoted elements
-   * @param strictQuotes sets if characters outside the quotes are ignored
-   */
-  public CSVReader(Reader reader, char separator, char quotechar, boolean strictQuotes) {
-    this(reader, separator, quotechar, CSVParser.DEFAULT_ESCAPE_CHARACTER, DEFAULT_SKIP_LINES,
-        strictQuotes);
-  }
-
-  /**
-   * Constructs CSVReader.
-   *
-   * @param reader    the reader to an underlying CSV source.
-   * @param separator the delimiter to use for separating entries
-   * @param quotechar the character to use for quoted elements
-   * @param escape    the character to use for escaping a separator or quote
-   */
-
-  public CSVReader(Reader reader, char separator, char quotechar, char escape) {
-    this(reader, separator, quotechar, escape, DEFAULT_SKIP_LINES, CSVParser.DEFAULT_STRICT_QUOTES);
-  }
-
-  /**
-   * Constructs CSVReader.
-   *
-   * @param reader    the reader to an underlying CSV source.
-   * @param separator the delimiter to use for separating entries
-   * @param quotechar the character to use for quoted elements
-   * @param line      the line number to skip for start reading
-   */
-  public CSVReader(Reader reader, char separator, char quotechar, int line) {
-    this(reader, separator, quotechar, CSVParser.DEFAULT_ESCAPE_CHARACTER, line,
-        CSVParser.DEFAULT_STRICT_QUOTES);
-  }
-
-  /**
-   * Constructs CSVReader.
-   *
-   * @param reader    the reader to an underlying CSV source.
-   * @param separator the delimiter to use for separating entries
-   * @param quotechar the character to use for quoted elements
-   * @param escape    the character to use for escaping a separator or quote
-   * @param line      the line number to skip for start reading
-   */
-  public CSVReader(Reader reader, char separator, char quotechar, char escape, int line) {
-    this(reader, separator, quotechar, escape, line, CSVParser.DEFAULT_STRICT_QUOTES);
-  }
-
-  /**
-   * Constructs CSVReader.
-   *
-   * @param reader       the reader to an underlying CSV source.
-   * @param separator    the delimiter to use for separating entries
-   * @param quotechar    the character to use for quoted elements
-   * @param escape       the character to use for escaping a separator or quote
-   * @param line         the line number to skip for start reading
-   * @param strictQuotes sets if characters outside the quotes are ignored
-   */
-  public CSVReader(Reader reader, char separator, char quotechar, char escape, int line,
-      boolean strictQuotes) {
-    this(reader, separator, quotechar, escape, line, strictQuotes,
-        CSVParser.DEFAULT_IGNORE_LEADING_WHITESPACE);
-  }
-
-  /**
-   * Constructs CSVReader with all data entered.
-   *
-   * @param reader                  the reader to an underlying CSV source.
-   * @param separator               the delimiter to use for separating entries
-   * @param quotechar               the character to use for quoted elements
-   * @param escape                  the character to use for escaping a separator or quote
-   * @param line                    the line number to skip for start reading
-   * @param strictQuotes            sets if characters outside the quotes are ignored
-   * @param ignoreLeadingWhiteSpace it true, parser should ignore white space before a quote
-   *                                in a field
-   */
-  public CSVReader(Reader reader, char separator, char quotechar, char escape, int line,
-      boolean strictQuotes, boolean ignoreLeadingWhiteSpace) {
-    this(reader, line,
-        new CSVParser(separator, quotechar, escape, strictQuotes, ignoreLeadingWhiteSpace));
-  }
-
-  /**
-   * Constructs CSVReader with all data entered.
-   *
-   * @param reader                  the reader to an underlying CSV source.
-   * @param separator               the delimiter to use for separating entries
-   * @param quotechar               the character to use for quoted elements
-   * @param escape                  the character to use for escaping a separator or quote
-   * @param line                    the line number to skip for start reading
-   * @param strictQuotes            sets if characters outside the quotes are ignored
-   * @param ignoreLeadingWhiteSpace if true, parser should ignore white space before a quote
-   *                                in a field
-   * @param keepCR                  if true the reader will keep carriage returns,
-   *                                otherwise it will discard them.
-   */
-  public CSVReader(Reader reader, char separator, char quotechar, char escape, int line,
-      boolean strictQuotes, boolean ignoreLeadingWhiteSpace, boolean keepCR) {
-    this(reader, line,
-        new CSVParser(separator, quotechar, escape, strictQuotes, ignoreLeadingWhiteSpace), keepCR,
-        DEFAULT_VERIFY_READER);
-  }
-
-  /**
-   * Constructs CSVReader with supplied CSVParser.
-   *
-   * @param reader    the reader to an underlying CSV source.
-   * @param line      the line number to skip for start reading
-   * @param csvParser the parser to use to parse input
-   */
-  public CSVReader(Reader reader, int line, CSVParser csvParser, boolean multiLine) {
-    this(reader, line, csvParser);
-    this.multiLine = multiLine;
-  }
-
-  public CSVReader(Reader reader, int line, CSVParser csvParser) {
-    this(reader, line, csvParser, DEFAULT_KEEP_CR, DEFAULT_VERIFY_READER);
-  }
-
-  /**
-   * Constructs CSVReader with supplied CSVParser.
-   *
-   * @param reader       the reader to an underlying CSV source.
-   * @param line         the line number to skip for start reading
-   * @param csvParser    the parser to use to parse input
-   * @param keepCR       true to keep carriage returns in data read, false otherwise
-   * @param verifyReader true to verify reader before each read, false otherwise
-   */
-  CSVReader(Reader reader, int line, CSVParser csvParser, boolean keepCR, boolean verifyReader) {
-    this.br = (reader instanceof BufferedReader ?
-        (BufferedReader) reader :
-        new BufferedReader(reader, 30720));
-    this.lineReader = new LineReader(br, keepCR);
-    this.skipLines = line;
-    this.parser = csvParser;
-    this.keepCR = keepCR;
-    this.verifyReader = verifyReader;
-  }
-
-  /**
-   * @return the CSVParser used by the reader.
-   */
-  public CSVParser getParser() {
-    return parser;
-  }
-
-  /**
-   * Returns the number of lines in the csv file to skip before processing.  This is
-   * useful when there is miscellaneous data at the beginning of a file.
-   *
-   * @return the number of lines in the csv file to skip before processing.
-   */
-  public int getSkipLines() {
-    return skipLines;
-  }
-
-  /**
-   * Returns if the reader will keep carriage returns found in data or remove them.
-   *
-   * @return true if reader will keep carriage returns, false otherwise.
-   */
-  public boolean keepCarriageReturns() {
-    return keepCR;
-  }
-
-  /**
-   * Reads the entire file into a List with each element being a String[] of
-   * tokens.
-   *
-   * @return a List of String[], with each String[] representing a line of the
-   * file.
-   * @throws IOException if bad things happen during the read
-   */
-  public List<String[]> readAll() throws IOException {
-
-    //CHECKSTYLE:OFF Approval No:Approval-V1R2C10_005
-    List<String[]> allElements = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-    while (hasNext) {
-      String[] nextLineAsTokens = readNext();
-      if (nextLineAsTokens != null) {
-        allElements.add(nextLineAsTokens);
-      }
-    }
-    return allElements;
-    //CHECKSTYLE:ON
-  }
-
-  /**
-   * Reads the next line from the buffer and converts to a string array.
-   *
-   * @return a string array with each comma-separated element as a separate
-   * entry.
-   * @throws IOException if bad things happen during the read
-   */
-  public String[] readNext() throws IOException {
-
-    if (this.multiLine) {
-      return readMultiLine();
-    } else {
-      return readSingleLine();
-    }
-  }
-
-  private String[] readSingleLine() throws IOException {
-    String[] result = null;
-    String nextLine = getNextLine();
-    try {
-      this.lineNum += 1L;
-      result = parser.parseLine(nextLine);
-    } catch (IOException e) {
-      if ("Un-terminated quoted field at end of CSV line".equals(e.getMessage())) {
-        badRecordslogger.addBadRecordsToBilder(new String[] { nextLine }, 1,
-            "Un-terminated quoted field at end of CSV line", null);
-        LOGGER.info("Found Un-terminated quote @ line [" + this.lineNum + "] : Skipping Line : "
-                + nextLine);
-        this.skippedLines += 1L;
-        result = readNext();
-      } else {
-        throw e;
-      }
-    }
-    if (null == nextLine) {
-      LOGGER.info("Total Number of Lines : " + --this.lineNum);
-      LOGGER.info("Number of Lines Skipped: " + this.skippedLines);
-      //            System.out.println("Total Number of Lines : "+ --this.lineNum);
-      //            System.out.println("Number of Lines Skipped: "+ this.skippedLines);
-    }
-    return result;
-  }
-
-  private String[] readMultiLine() throws IOException {
-    int linesread = 0;
-    String[] result = null;
-    String firstLine = null;
-    do {
-      this.lineNum += 1L;
-      linesread++;
-      if (linesread == 2) {
-        br.mark(12000);
-      }
-      String nextLine = getNextLine();
-      if (!hasNext) {
-        LOGGER.info("Total Number of Lines : " + --this.lineNum);
-        LOGGER.info("Number of Lines Skipped: " + this.skippedLines);
-        //             System.out.println("Total Number of Lines : "+ --this.lineNum);
-        //             System.out.println("Number of Lines Skipped: "+ this.skippedLines);
-        return result; // should throw if still pending?
-      }
-      try {
-        String[] r = parser.parseLineMulti(nextLine);
-        if (r.length > 0) {
-          if (result == null) {
-            result = r;
-          } else {
-            result = combineResultsFromMultipleReads(result, r);
-          }
-        }
-      } catch (IOException e) {
-        if ("Un-terminated quoted field after 10000 characters".equals(e.getMessage())) {
-          LOGGER.info("Un-terminated quoted field found after 10000 characters in MultiLine "
-                  + "(No. Of Line searched : " + linesread + " ) starting from Line :" + (
-                  this.lineNum - linesread + 1));
-          LOGGER.info("Skipped Line Info : " + firstLine);
-          parser.setPending(null);
-          this.skippedLines += 1;
-          this.lineNum += (1 - linesread);
-          if (linesread > 1) {
-            br.reset();
-          }
-          int resLength = result != null ? result.length : 0;
-          badRecordslogger.addBadRecordsToBilder(result, resLength,
-              "Un-terminated quoted field after 10000 characters", null);
-          result = readNext();
-        } else {
-          throw e;
-        }
-      }
-      if (linesread == 1) {
-        firstLine = nextLine;
-      }
-      //            String[] r = parser.parseLine(nextLine);
-    } while (parser.isPending());
-    return result;
-  }
-
-  /**
-   * For multi line records this method combines the current result with the result
-   * from previous read(s).
-   *
-   * @param buffer   - previous data read for this record
-   * @param lastRead - latest data read for this record.
-   * @return String array with union of the buffer and lastRead arrays.
-   */
-  private String[] combineResultsFromMultipleReads(String[] buffer, String[] lastRead) {
-    String[] t = new String[buffer.length + lastRead.length];
-    System.arraycopy(buffer, 0, t, 0, buffer.length);
-    System.arraycopy(lastRead, 0, t, buffer.length, lastRead.length);
-    return t;
-  }
-
-  /**
-   * Reads the next line from the file.
-   *
-   * @return the next line from the file without trailing newline
-   * @throws IOException if bad things happen during the read
-   */
-  public String getNextLine() throws IOException {
-    if (isClosed()) {
-      hasNext = false;
-      return null;
-    }
-
-    if (!this.linesSkiped) {
-      for (int i = 0; i < skipLines; i++) {
-        lineReader.readLine();
-      }
-      this.linesSkiped = true;
-    }
-    String nextLine = lineReader.readLine();
-    if (nextLine == null) {
-      hasNext = false;
-    }
-    return hasNext ? nextLine : null;
-  }
-
-  /**
-   * Checks to see if the file is closed.
-   *
-   * @return true if the reader can no longer be read from.
-   */
-  private boolean isClosed() {
-    if (!verifyReader) {
-      return false;
-    }
-    try {
-      return !br.ready();
-    } catch (IOException e) {
-      return true;
-    }
-  }
-
-  /**
-   * Closes the underlying reader.
-   *
-   * @throws IOException if the close fails
-   */
-  public void close() throws IOException {
-    br.close();
-  }
-
-  /**
-   * Creates an Iterator for processing the csv data.
-   *
-   * @return an String[] iterator.
-   */
-  public Iterator<String[]> iterator() {
-    try {
-      return new CSVIterator(this);
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-  }
-
-  /**
-   * Returns if the CSVReader will verify the reader before each read.
-   * By default the value is true which is the functionality for version 3.0.
-   * If set to false the reader is always assumed ready to read - this is the functionality
-   * for version 2.4 and before.
-   * The reason this method was needed was that certain types of Readers would return
-   * false for its ready() method until a read was done (namely readers created using Channels).
-   * This caused opencsv not to read from those readers.
-   *
-   * @return true if CSVReader will verify the reader before reads.  False otherwise.
-   * @link https://sourceforge.net/p/opencsv/bugs/108/
-   */
-  public boolean verifyReader() {
-    return this.verifyReader;
-  }
-
-  public void setBadRecordsLogger(BadRecordslogger badRecordslogger) {
-    this.badRecordslogger = badRecordslogger;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVWriter.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVWriter.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVWriter.java
deleted file mode 100644
index 68dbd59..0000000
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/CSVWriter.java
+++ /dev/null
@@ -1,396 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.carbondata.spark.partition.reader;
-
-/**
- * Copyright 2005 Bytecode Pty Ltd.
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.Closeable;
-import java.io.Flushable;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.io.Writer;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.List;
-
-import org.apache.carbondata.common.logging.LogService;
-import org.apache.carbondata.common.logging.LogServiceFactory;
-
-/**
- * A very simple CSV writer released under a commercial-friendly license.
- *
- * @author Glen Smith
- */
-public class CSVWriter implements Closeable, Flushable {
-
-  public static final int INITIAL_STRING_SIZE = 128;
-  /**
-   * The character used for escaping quotes.
-   */
-  public static final char DEFAULT_ESCAPE_CHARACTER = '"';
-  /**
-   * The default separator to use if none is supplied to the constructor.
-   */
-  public static final char DEFAULT_SEPARATOR = ',';
-  /**
-   * The default quote character to use if none is supplied to the
-   * constructor.
-   */
-  public static final char DEFAULT_QUOTE_CHARACTER = '"';
-  /**
-   * The quote constant to use when you wish to suppress all quoting.
-   */
-  public static final char NO_QUOTE_CHARACTER = '\u0000';
-  /**
-   * The escape constant to use when you wish to suppress all escaping.
-   */
-  public static final char NO_ESCAPE_CHARACTER = '\u0000';
-  /**
-   * Default line terminator uses platform encoding.
-   */
-  public static final String DEFAULT_LINE_END = "\n";
-
-  public static final String CARRIAGE_RETURN = "\r";
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CSVWriter.class.getName());
-
-  private Writer rawWriter;
-  private PrintWriter pw;
-  private char separator;
-  private char quotechar;
-  private char escapechar;
-  private String lineEnd;
-  private ResultSetHelper resultService = new ResultSetHelperService();
-
-  /**
-   * Constructs CSVWriter using a comma for the separator.
-   *
-   * @param writer the writer to an underlying CSV source.
-   */
-  public CSVWriter(Writer writer) {
-    this(writer, DEFAULT_SEPARATOR);
-  }
-
-  /**
-   * Constructs CSVWriter with supplied separator.
-   *
-   * @param writer    the writer to an underlying CSV source.
-   * @param separator the delimiter to use for separating entries.
-   */
-  public CSVWriter(Writer writer, char separator) {
-    this(writer, separator, DEFAULT_QUOTE_CHARACTER);
-  }
-
-  /**
-   * Constructs CSVWriter with supplied separator and quote char.
-   *
-   * @param writer    the writer to an underlying CSV source.
-   * @param separator the delimiter to use for separating entries
-   * @param quotechar the character to use for quoted elements
-   */
-  public CSVWriter(Writer writer, char separator, char quotechar) {
-    this(writer, separator, quotechar, DEFAULT_ESCAPE_CHARACTER);
-  }
-
-  /**
-   * Constructs CSVWriter with supplied separator and quote char.
-   *
-   * @param writer     the writer to an underlying CSV source.
-   * @param separator  the delimiter to use for separating entries
-   * @param quotechar  the character to use for quoted elements
-   * @param escapechar the character to use for escaping quotechars or escapechars
-   */
-  public CSVWriter(Writer writer, char separator, char quotechar, char escapechar) {
-    this(writer, separator, quotechar, escapechar, DEFAULT_LINE_END);
-  }
-
-  /**
-   * Constructs CSVWriter with supplied separator and quote char.
-   *
-   * @param writer    the writer to an underlying CSV source.
-   * @param separator the delimiter to use for separating entries
-   * @param quotechar the character to use for quoted elements
-   * @param lineEnd   the line feed terminator to use
-   */
-  public CSVWriter(Writer writer, char separator, char quotechar, String lineEnd) {
-    this(writer, separator, quotechar, DEFAULT_ESCAPE_CHARACTER, lineEnd);
-  }
-
-  /**
-   * Constructs CSVWriter with supplied separator, quote char, escape char and line ending.
-   *
-   * @param writer     the writer to an underlying CSV source.
-   * @param separator  the delimiter to use for separating entries
-   * @param quotechar  the character to use for quoted elements
-   * @param escapechar the character to use for escaping quotechars or escapechars
-   * @param lineEnd    the line feed terminator to use
-   */
-  public CSVWriter(Writer writer, char separator, char quotechar, char escapechar, String lineEnd) {
-    this.rawWriter = writer;
-    this.pw = new PrintWriter(writer);
-    this.separator = separator;
-    this.quotechar = quotechar;
-    this.escapechar = escapechar;
-    this.lineEnd = lineEnd;
-  }
-
-  /**
-   * Writes the entire list to a CSV file. The list is assumed to be a
-   * String[]
-   *
-   * @param allLines         a List of String[], with each String[] representing a line of
-   *                         the file.
-   * @param applyQuotesToAll true if all values are to be quoted.  false if quotes only
-   *                         to be applied to values which contain the separator, escape,
-   *                         quote or new line characters.
-   */
-  public void writeAll(List<String[]> allLines, boolean applyQuotesToAll) {
-    for (String[] line : allLines) {
-      writeNext(line, applyQuotesToAll);
-    }
-  }
-
-  /**
-   * Writes the entire list to a CSV file. The list is assumed to be a
-   * String[]
-   *
-   * @param allLines a List of String[], with each String[] representing a line of
-   *                 the file.
-   */
-  public void writeAll(List<String[]> allLines) {
-    for (String[] line : allLines) {
-      writeNext(line);
-    }
-  }
-
-  /**
-   * Writes the column names.
-   *
-   * @param rs - ResultSet containing column names.
-   * @throws SQLException - thrown by ResultSet::getColumnNames
-   */
-  protected void writeColumnNames(ResultSet rs) throws SQLException {
-
-    writeNext(resultService.getColumnNames(rs));
-  }
-
-  /**
-   * Writes the entire ResultSet to a CSV file.
-   * The caller is responsible for closing the ResultSet.
-   *
-   * @param rs                 the result set to write
-   * @param includeColumnNames true if you want column names in the output, false otherwise
-   * @throws IOException  thrown by getColumnValue
-   * @throws SQLException thrown by getColumnValue
-   */
-  public void writeAll(ResultSet rs, boolean includeColumnNames) throws SQLException, IOException {
-    writeAll(rs, includeColumnNames, false);
-  }
-
-  /**
-   * Writes the entire ResultSet to a CSV file.
-   * The caller is responsible for closing the ResultSet.
-   *
-   * @param rs                 the Result set to write.
-   * @param includeColumnNames include the column names in the output.
-   * @param trim               remove spaces from the data before writing.
-   * @throws IOException  thrown by getColumnValue
-   * @throws SQLException thrown by getColumnValue
-   */
-  public void writeAll(ResultSet rs, boolean includeColumnNames, boolean trim)
-      throws SQLException, IOException {
-
-    if (includeColumnNames) {
-      writeColumnNames(rs);
-    }
-
-    while (rs.next()) {
-      writeNext(resultService.getColumnValues(rs, trim));
-    }
-  }
-
-  /**
-   * Writes the next line to the file.
-   *
-   * @param nextLine         a string array with each comma-separated element as a separate
-   *                         entry.
-   * @param applyQuotesToAll true if all values are to be quoted.  false applies quotes only
-   *                         to values which contain the separator, escape, quote or new line
-   *                         characters.
-   */
-  public void writeNext(String[] nextLine, boolean applyQuotesToAll) {
-
-    if (nextLine == null) {
-      return;
-    }
-
-    StringBuilder sb = new StringBuilder(INITIAL_STRING_SIZE);
-    for (int i = 0; i < nextLine.length; i++) {
-
-      if (i != 0) {
-        sb.append(separator);
-      }
-
-      String nextElement = nextLine[i];
-
-      if (nextElement == null) {
-        continue;
-      }
-
-      Boolean stringContainsSpecialCharacters = stringContainsSpecialCharacters(nextElement);
-
-      if ((applyQuotesToAll || stringContainsSpecialCharacters)
-          && quotechar != NO_QUOTE_CHARACTER) {
-        sb.append(quotechar);
-      }
-
-      if (stringContainsSpecialCharacters) {
-        sb.append(processLine(nextElement));
-      } else {
-        sb.append(nextElement);
-      }
-
-      if ((applyQuotesToAll || stringContainsSpecialCharacters)
-          && quotechar != NO_QUOTE_CHARACTER) {
-        sb.append(quotechar);
-      }
-    }
-
-    sb.append(lineEnd);
-    pw.write(sb.toString());
-  }
-
-  /**
-   * Writes the next line to the file.
-   *
-   * @param nextLine a string array with each comma-separated element as a separate
-   *                 entry.
-   */
-  public void writeNext(String[] nextLine) {
-    writeNext(nextLine, true);
-  }
-
-  /**
-   * checks to see if the line contains special characters.
-   *
-   * @param line - element of data to check for special characters.
-   * @return true if the line contains the quote, escape, separator, newline or return.
-   */
-  private boolean stringContainsSpecialCharacters(String line) {
-    return line.indexOf(quotechar) != -1 || line.indexOf(escapechar) != -1
-        || line.indexOf(separator) != -1 || line.contains(DEFAULT_LINE_END) || line
-        .contains(CARRIAGE_RETURN);
-  }
-
-  /**
-   * Processes all the characters in a line.
-   *
-   * @param nextElement - element to process.
-   * @return a StringBuilder with the elements data.
-   */
-  protected StringBuilder processLine(String nextElement) {
-    StringBuilder sb = new StringBuilder(INITIAL_STRING_SIZE);
-    for (int j = 0; j < nextElement.length(); j++) {
-      char nextChar = nextElement.charAt(j);
-      processCharacter(sb, nextChar);
-    }
-
-    return sb;
-  }
-
-  /**
-   * Appends the character to the StringBuilder adding the escape character if needed.
-   *
-   * @param sb       - StringBuffer holding the processed character.
-   * @param nextChar - character to process
-   */
-  private void processCharacter(StringBuilder sb, char nextChar) {
-    if (escapechar != NO_ESCAPE_CHARACTER && (nextChar == quotechar || nextChar == escapechar)) {
-      sb.append(escapechar).append(nextChar);
-    } else {
-      sb.append(nextChar);
-    }
-  }
-
-  /**
-   * Flush underlying stream to writer.
-   *
-   * @throws IOException if bad things happen
-   */
-  public void flush() throws IOException {
-
-    pw.flush();
-
-  }
-
-  /**
-   * Close the underlying stream writer flushing any buffered content.
-   *
-   * @throws IOException if bad things happen
-   */
-  public void close() throws IOException {
-    flush();
-    pw.close();
-    rawWriter.close();
-  }
-
-  /**
-   * Checks to see if the there has been an error in the printstream.
-   *
-   * @return <code>true</code> if the print stream has encountered an error,
-   * either on the underlying output stream or during a format
-   * conversion.
-   */
-  public boolean checkError() {
-    return pw.checkError();
-  }
-
-  /**
-   * Sets the result service.
-   *
-   * @param resultService - the ResultSetHelper
-   */
-  public void setResultService(ResultSetHelper resultService) {
-    this.resultService = resultService;
-  }
-
-  /**
-   * flushes the writer without throwing any exceptions.
-   */
-  public void flushQuietly() {
-    try {
-      flush();
-    } catch (IOException e) {
-      LOGGER.debug("Error while flushing");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be1675f4/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/LineReader.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/LineReader.java b/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/LineReader.java
deleted file mode 100644
index 3ee6228..0000000
--- a/integration/spark/src/main/java/org/apache/carbondata/spark/partition/reader/LineReader.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.carbondata.spark.partition.reader;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-
-/**
- * This class was created for issue #106 (https://sourceforge.net/p/opencsv/bugs/106/) where
- * carriage returns were being removed.  This class allows the user to determine if they wish to
- * keep or remove them from the data being read.
- * Created by scott on 2/19/15.
- */
-
-public class LineReader {
-  private BufferedReader reader;
-  private boolean keepCarriageReturns;
-
-  /**
-   * LineReader constructor.
-   *
-   * @param reader              - Reader that data will be read from.
-   * @param keepCarriageReturns - true if carriage returns should remain in the data, false
-   *                            to remove them.
-   */
-  public LineReader(BufferedReader reader, boolean keepCarriageReturns) {
-    this.reader = reader;
-    this.keepCarriageReturns = keepCarriageReturns;
-  }
-
-  /**
-   * Reads the next line from the Reader.
-   *
-   * @return - Line read from reader.
-   * @throws IOException - on error from BufferedReader
-   */
-  public String readLine() throws IOException {
-    return keepCarriageReturns ? readUntilNewline() : reader.readLine();
-  }
-
-  private String readUntilNewline() throws IOException {
-    StringBuilder sb = new StringBuilder(CSVParser.INITIAL_READ_SIZE);
-    for (int c = reader.read();
-         c > -1 && c != '\n';
-         c = reader.read()) {      //CHECKSTYLE:OFF    Approval No:Approval-V1R2C10_006
-      sb.append((char) c);
-    }//CHECKSTYLE:ON
-
-    return sb.length() > 0 ? sb.toString() : null;
-  }
-}


[39/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-180]give proper error message when dataloading with wrong delimiter or fileheader This closes #97

Posted by ra...@apache.org.
[CARBONDATA-180]give proper error message when dataloading with wrong delimiter or fileheader This closes #97


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

Branch: refs/heads/branch-0.1
Commit: 30548bae8b7e553bd78cef1950b1556b6f70bf4e
Parents: 0d14af2
Author: ravipesala <ra...@gmail.com>
Authored: Tue Aug 30 19:52:59 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:32:01 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[50/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-251] making the dataload+auto compaction as blocking call . This closes #168

Posted by ra...@apache.org.
[CARBONDATA-251] making the dataload+auto compaction as blocking call . This closes #168


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

Branch: refs/heads/branch-0.1
Commit: 920b2deea42318a35de4f45afa290fe4213c27c3
Parents: fe274a9
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sun Sep 18 03:13:35 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:56:10 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[33/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-201]add comment option This closes #118

Posted by ra...@apache.org.
[CARBONDATA-201]add comment option This closes #118


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

Branch: refs/heads/branch-0.1
Commit: 8f8933ecd64fe6814bbf8a0ee44787584bcd94aa
Parents: 868ada5
Author: chenliang613 <ch...@apache.org>
Authored: Wed Sep 7 20:08:07 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:08:56 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[26/50] [abbrv] incubator-carbondata git commit: Fixed out of memory issue during query execution, as invalid segments are not getting deleted from Btree

Posted by ra...@apache.org.
Fixed out of memory issue during query execution, as invalid segments are not getting deleted from Btree


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

Branch: refs/heads/branch-0.1
Commit: 57ac4a5c61cd775d1d0c902a4fff2563aa03dc43
Parents: 14029e2
Author: kumarvishal <ku...@gmail.com>
Authored: Thu Sep 15 14:11:00 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:00:26 2016 +0530

----------------------------------------------------------------------
 .../core/carbon/datastore/BlockIndexStore.java  |  60 +++++++++-
 .../carbon/datastore/SegmentTaskIndexStore.java |   6 +-
 .../apache/carbondata/core/util/CarbonUtil.java |  21 ++++
 .../executor/impl/AbstractQueryExecutor.java    |  13 +-
 .../carbondata/scan/model/QueryModel.java       |  16 +++
 .../carbondata/hadoop/CarbonInputFormat.java    | 120 +++++++++++--------
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  20 +++-
 .../CompactionSystemLockFeatureTest.scala       |   4 +-
 .../DataCompactionCardinalityBoundryTest.scala  |   2 +-
 .../datacompaction/DataCompactionLockTest.scala |   2 +-
 .../DataCompactionNoDictionaryTest.scala        |   2 +-
 .../datacompaction/DataCompactionTest.scala     |   4 +-
 .../MajorCompactionIgnoreInMinorTest.scala      |   4 +-
 .../MajorCompactionStopsAfterCompaction.scala   |   4 +-
 .../lcm/status/SegmentStatusManager.java        |  80 ++++++++-----
 15 files changed, 247 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
index 07815c0..4a36373 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/BlockIndexStore.java
@@ -20,6 +20,7 @@ package org.apache.carbondata.core.carbon.datastore;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
@@ -62,6 +63,12 @@ public class BlockIndexStore {
   private Map<AbsoluteTableIdentifier, Map<TableBlockInfo, AbstractIndex>> tableBlocksMap;
 
   /**
+   * map to maintain segment id to block info map, this map will be used to
+   * while removing the block from memory when segment is compacted or deleted
+   */
+  private Map<AbsoluteTableIdentifier, Map<String, List<TableBlockInfo>>> segmentIdToBlockListMap;
+
+  /**
    * map of block info to lock object map, while loading the btree this will be filled
    * and removed after loading the tree for that particular block info, this will be useful
    * while loading the tree concurrently so only block level lock will be applied another
@@ -83,6 +90,7 @@ public class BlockIndexStore {
     tableLockMap = new ConcurrentHashMap<AbsoluteTableIdentifier, Object>(
         CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     blockInfoLock = new ConcurrentHashMap<TableBlockInfo, Object>();
+    segmentIdToBlockListMap = new ConcurrentHashMap<>();
   }
 
   /**
@@ -129,6 +137,7 @@ public class BlockIndexStore {
         tableBlockMapTemp = new ConcurrentHashMap<TableBlockInfo, AbstractIndex>();
         tableBlocksMap.put(absoluteTableIdentifier, tableBlockMapTemp);
       }
+      fillSegmentIdToTableInfoMap(tableBlocksInfos, absoluteTableIdentifier);
     }
     AbstractIndex tableBlock = null;
     List<Future<AbstractIndex>> blocksList = new ArrayList<Future<AbstractIndex>>();
@@ -189,6 +198,32 @@ public class BlockIndexStore {
   }
 
   /**
+   * Below method will be used to fill segment id to its block mapping map.
+   * it will group all the table block info based on segment id and it will fill
+   *
+   * @param tableBlockInfos         table block infos
+   * @param absoluteTableIdentifier absolute table identifier
+   */
+  private void fillSegmentIdToTableInfoMap(List<TableBlockInfo> tableBlockInfos,
+      AbsoluteTableIdentifier absoluteTableIdentifier) {
+    Map<String, List<TableBlockInfo>> map = segmentIdToBlockListMap.get(absoluteTableIdentifier);
+    if (null == map) {
+      map = new ConcurrentHashMap<String, List<TableBlockInfo>>();
+      segmentIdToBlockListMap.put(absoluteTableIdentifier, map);
+    }
+    for (TableBlockInfo info : tableBlockInfos) {
+      List<TableBlockInfo> tempTableBlockInfos = map.get(info.getSegmentId());
+      if (null == tempTableBlockInfos) {
+        tempTableBlockInfos = new ArrayList<>();
+        map.put(info.getSegmentId(), tempTableBlockInfos);
+      }
+      if (!tempTableBlockInfos.contains(info)) {
+        tempTableBlockInfos.add(info);
+      }
+    }
+  }
+
+  /**
    * Below method will be used to fill the loaded blocks to the array
    * which will be used for query execution
    *
@@ -246,10 +281,10 @@ public class BlockIndexStore {
    * deletion of some of the blocks in case of retention or may be some other
    * scenario
    *
-   * @param removeTableBlocksInfos  blocks to be removed
+   * @param segmentsToBeRemoved     list of segments to be removed
    * @param absoluteTableIdentifier absolute table identifier
    */
-  public void removeTableBlocks(List<TableBlockInfo> removeTableBlocksInfos,
+  public void removeTableBlocks(List<String> segmentsToBeRemoved,
       AbsoluteTableIdentifier absoluteTableIdentifier) {
     // get the lock object if lock object is not present then it is not
     // loaded at all
@@ -260,11 +295,26 @@ public class BlockIndexStore {
     }
     Map<TableBlockInfo, AbstractIndex> map = tableBlocksMap.get(absoluteTableIdentifier);
     // if there is no loaded blocks then return
-    if (null == map) {
+    if (null == map || map.isEmpty()) {
       return;
     }
-    for (TableBlockInfo blockInfos : removeTableBlocksInfos) {
-      map.remove(blockInfos);
+    Map<String, List<TableBlockInfo>> segmentIdToBlockInfoMap =
+        segmentIdToBlockListMap.get(absoluteTableIdentifier);
+    if (null == segmentIdToBlockInfoMap || segmentIdToBlockInfoMap.isEmpty()) {
+      return;
+    }
+    synchronized (lockObject) {
+      for (String segmentId : segmentsToBeRemoved) {
+        List<TableBlockInfo> tableBlockInfoList = segmentIdToBlockInfoMap.remove(segmentId);
+        if (null == tableBlockInfoList) {
+          continue;
+        }
+        Iterator<TableBlockInfo> tableBlockInfoIterator = tableBlockInfoList.iterator();
+        while (tableBlockInfoIterator.hasNext()) {
+          TableBlockInfo info = tableBlockInfoIterator.next();
+          map.remove(info);
+        }
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/core/src/main/java/org/apache/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java
index 50d462a..e2218a8 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java
@@ -141,8 +141,8 @@ public class SegmentTaskIndexStore {
           synchronized (segmentLoderLockObject) {
             taskIdToSegmentIndexMap = tableSegmentMapTemp.get(segmentId);
             if (null == taskIdToSegmentIndexMap) {
-              // creating a map of take if to table segment
-              taskIdToSegmentIndexMap = new HashMap<String, AbstractIndex>();
+              // creating a map of task id to table segment
+              taskIdToSegmentIndexMap = new ConcurrentHashMap<String, AbstractIndex>();
               Iterator<Entry<String, List<TableBlockInfo>>> iterator =
                   taskIdToTableBlockInfoMap.entrySet().iterator();
               while (iterator.hasNext()) {
@@ -256,7 +256,7 @@ public class SegmentTaskIndexStore {
   private Map<String, List<TableBlockInfo>> mappedAndGetTaskIdToTableBlockInfo(
       Map<String, List<TableBlockInfo>> segmentToTableBlocksInfos) {
     Map<String, List<TableBlockInfo>> taskIdToTableBlockInfoMap =
-        new HashMap<String, List<TableBlockInfo>>();
+        new ConcurrentHashMap<String, List<TableBlockInfo>>();
     Iterator<Entry<String, List<TableBlockInfo>>> iterator =
         segmentToTableBlocksInfos.entrySet().iterator();
     while (iterator.hasNext()) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/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 82c515c..5168208 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
@@ -1431,5 +1431,26 @@ public final class CarbonUtil {
     return builder.toString();
   }
 
+  /**
+   * Below method will be used to get the list of segment in
+   * comma separated string format
+   *
+   * @param segmentList
+   * @return comma separated segment string
+   */
+  public static String getSegmentString(List<String> segmentList) {
+    if (segmentList.isEmpty()) {
+      return "";
+    }
+    StringBuilder segmentStringbuilder = new StringBuilder();
+    for (int i = 0; i < segmentList.size() - 1; i++) {
+      String segmentNo = segmentList.get(i);
+      segmentStringbuilder.append(segmentNo);
+      segmentStringbuilder.append(",");
+    }
+    segmentStringbuilder.append(segmentList.get(segmentList.size() - 1));
+    return segmentStringbuilder.toString();
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
index dab8a23..c31824f 100644
--- a/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
@@ -18,12 +18,7 @@
  */
 package org.apache.carbondata.scan.executor.impl;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
+import java.util.*;
 
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -101,8 +96,12 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // query execution
     Collections.sort(queryModel.getTableBlockInfos());
     // get the table blocks
+    BlockIndexStore blockLoaderInstance = BlockIndexStore.getInstance();
+    // remove the invalid table blocks, block which is deleted or compacted
+    blockLoaderInstance.removeTableBlocks(queryModel.getInvalidSegmentIds(),
+        queryModel.getAbsoluteTableIdentifier());
     try {
-      queryProperties.dataBlocks = BlockIndexStore.getInstance()
+      queryProperties.dataBlocks = blockLoaderInstance
           .loadAndGetBlocks(queryModel.getTableBlockInfos(),
               queryModel.getAbsoluteTableIdentifier());
     } catch (IndexBuilderException e) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/core/src/main/java/org/apache/carbondata/scan/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/model/QueryModel.java b/core/src/main/java/org/apache/carbondata/scan/model/QueryModel.java
index 81eb728..1c819a2 100644
--- a/core/src/main/java/org/apache/carbondata/scan/model/QueryModel.java
+++ b/core/src/main/java/org/apache/carbondata/scan/model/QueryModel.java
@@ -132,6 +132,13 @@ public class QueryModel implements Serializable {
 
   private QueryStatisticsRecorder statisticsRecorder;
 
+  /**
+   * Invalid table blocks, which need to be removed from
+   * memory, invalid blocks can be segment which are deleted
+   * or compacted
+   */
+  private List<String> invalidSegmentIds;
+
   public QueryModel() {
     tableBlockInfos = new ArrayList<TableBlockInfo>();
     queryDimension = new ArrayList<QueryDimension>();
@@ -139,6 +146,7 @@ public class QueryModel implements Serializable {
     sortDimension = new ArrayList<QueryDimension>();
     sortOrder = new byte[0];
     paritionColumns = new ArrayList<String>();
+    invalidSegmentIds = new ArrayList<>();
   }
 
   public static QueryModel createModel(AbsoluteTableIdentifier absoluteTableIdentifier,
@@ -504,4 +512,12 @@ public class QueryModel implements Serializable {
   public void setStatisticsRecorder(QueryStatisticsRecorder statisticsRecorder) {
     this.statisticsRecorder = statisticsRecorder;
   }
+
+  public List<String> getInvalidSegmentIds() {
+    return invalidSegmentIds;
+  }
+
+  public void setInvalidSegmentIds(List<String> invalidSegmentIds) {
+    this.invalidSegmentIds = invalidSegmentIds;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
index da697ad..8fa1bb6 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
@@ -54,6 +54,7 @@ import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsRecorder
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
 import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
 import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport;
 import org.apache.carbondata.hadoop.readsupport.impl.DictionaryDecodedReadSupportImpl;
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil;
@@ -93,6 +94,9 @@ import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.util.StringUtils;
 
 
+
+
+
 /**
  * Carbon Input format class representing one carbon table
  */
@@ -236,21 +240,29 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
   /**
    * Set List of segments to access
    */
-  private void setSegmentsToAccess(Configuration configuration, List<String> segmentNosList) {
-
-    //serialize to comma separated string
-    StringBuilder stringSegmentsBuilder = new StringBuilder();
-    for (int i = 0; i < segmentNosList.size(); i++) {
-      String segmentNo = segmentNosList.get(i);
-      stringSegmentsBuilder.append(segmentNo);
-      if (i < segmentNosList.size() - 1) {
-        stringSegmentsBuilder.append(",");
-      }
-    }
-    configuration.set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, stringSegmentsBuilder.toString());
+  public static void setSegmentsToAccess(Configuration configuration, List<String> validSegments) {
+    configuration
+        .set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, CarbonUtil.getSegmentString(validSegments));
   }
 
   /**
+   * Below method will be used to set the segments details if
+   * segments are not added in the configuration
+   *
+   * @param job
+   * @param absoluteTableIdentifier
+   * @throws IOException
+   */
+  private void addSegmentsIfEmpty(JobContext job, AbsoluteTableIdentifier absoluteTableIdentifier)
+      throws IOException {
+    if (getSegmentsFromConfiguration(job).length == 0) {
+      // Get the valid segments from the carbon store.
+      SegmentStatusManager.ValidAndInvalidSegmentsInfo validAndInvalidSegments =
+          new SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments();
+      setSegmentsToAccess(job.getConfiguration(), validAndInvalidSegments.getValidSegments());
+    }
+  }
+  /**
    * {@inheritDoc}
    * Configurations FileInputFormat.INPUT_DIR
    * are used to get table path to read.
@@ -263,25 +275,17 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     try {
       CarbonTable carbonTable = getCarbonTable(job.getConfiguration());
       Object filterPredicates = getFilterPredicates(job.getConfiguration());
-      if (getValidSegments(job).length == 0) {
-        // Get the valid segments from the carbon store.
-        SegmentStatusManager.ValidSegmentsInfo validSegments =
-            new SegmentStatusManager(getAbsoluteTableIdentifier(job.getConfiguration()))
-                .getValidSegments();
-        if (validSegments.listOfValidSegments.isEmpty()) {
-          return new ArrayList<InputSplit>();
-        }
-        setSegmentsToAccess(job.getConfiguration(), validSegments.listOfValidSegments);
-      }
-
+      AbsoluteTableIdentifier absoluteTableIdentifier =
+          getAbsoluteTableIdentifier(job.getConfiguration());
+      addSegmentsIfEmpty(job, absoluteTableIdentifier);
       if (filterPredicates == null) {
         return getSplitsNonFilter(job);
       } else {
         if (filterPredicates instanceof Expression) {
           //process and resolve the expression.
           CarbonInputFormatUtil.processFilterExpression((Expression) filterPredicates, carbonTable);
-          return getSplits(job, CarbonInputFormatUtil.resolveFilter((Expression) filterPredicates,
-              getAbsoluteTableIdentifier(job.getConfiguration())));
+          return getSplits(job, CarbonInputFormatUtil
+              .resolveFilter((Expression) filterPredicates, absoluteTableIdentifier));
         } else {
           //It means user sets already resolved expression.
           return getSplits(job, (FilterResolverIntf) filterPredicates);
@@ -339,7 +343,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
         getAbsoluteTableIdentifier(job.getConfiguration());
 
     //for each segment fetch blocks matching filter in Driver BTree
-    for (String segmentNo : getValidSegments(job)) {
+    for (String segmentNo : getSegmentsFromConfiguration(job)) {
       List<DataRefNode> dataRefNodes =
           getDataBlocksOfSegment(job, filterExpressionProcessor, absoluteTableIdentifier,
               filterResolver, segmentNo);
@@ -365,11 +369,8 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     long rowCount = 0;
     AbsoluteTableIdentifier absoluteTableIdentifier =
         getAbsoluteTableIdentifier(job.getConfiguration());
-    SegmentStatusManager.ValidSegmentsInfo validSegments =
-        new SegmentStatusManager(getAbsoluteTableIdentifier(job.getConfiguration()))
-            .getValidSegments();
-    setSegmentsToAccess(job.getConfiguration(), validSegments.listOfValidSegments);
     // no of core to load the blocks in driver
+    addSegmentsIfEmpty(job, absoluteTableIdentifier);
     int numberOfCores = CarbonCommonConstants.NUMBER_OF_CORE_TO_LOAD_DRIVER_SEGMENT_DEFAULT_VALUE;
     try {
       numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
@@ -382,7 +383,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     List<Future<Map<String, AbstractIndex>>> loadedBlocks =
         new ArrayList<Future<Map<String, AbstractIndex>>>();
     //for each segment fetch blocks matching filter in Driver BTree
-    for (String segmentNo : getValidSegments(job)) {
+    for (String segmentNo : getSegmentsFromConfiguration(job)) {
       // submitting the task
       loadedBlocks
           .add(threadPool.submit(new BlocksLoaderThread(job, absoluteTableIdentifier, segmentNo)));
@@ -491,6 +492,39 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     return resultFilterredBlocks;
   }
 
+  /**
+   * Below method will be used to get the table block info
+   *
+   * @param job                     job context
+   * @param absoluteTableIdentifier absolute table identifier
+   * @param segmentId               number of segment id
+   * @return list of table block
+   * @throws IOException
+   */
+  private List<TableBlockInfo> getTableBlockInfo(JobContext job,
+      AbsoluteTableIdentifier absoluteTableIdentifier, String segmentId) throws IOException {
+    // List<FileStatus> fileStatusList = new LinkedList<FileStatus>();
+    List<TableBlockInfo> tableBlockInfoList = new ArrayList<TableBlockInfo>();
+    // getFileStatusOfSegments(job, new int[]{ segmentId }, fileStatusList);
+
+    // get file location of all files of given segment
+    JobContext newJob =
+        new JobContextImpl(new Configuration(job.getConfiguration()), job.getJobID());
+    newJob.getConfiguration().set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, segmentId + "");
+
+    // identify table blocks
+    for (InputSplit inputSplit : getSplitsInternal(newJob)) {
+      CarbonInputSplit carbonInputSplit = (CarbonInputSplit) inputSplit;
+      BlockletInfos blockletInfos = new BlockletInfos(carbonInputSplit.getNumberOfBlocklets(), 0,
+          carbonInputSplit.getNumberOfBlocklets());
+      tableBlockInfoList.add(
+          new TableBlockInfo(carbonInputSplit.getPath().toString(), carbonInputSplit.getStart(),
+              segmentId, carbonInputSplit.getLocations(), carbonInputSplit.getLength(),
+              blockletInfos));
+    }
+    return tableBlockInfoList;
+  }
+
   private Map<String, AbstractIndex> getSegmentAbstractIndexs(JobContext job,
       AbsoluteTableIdentifier absoluteTableIdentifier, String segmentId)
       throws IOException, IndexBuilderException {
@@ -500,25 +534,10 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
     // if segment tree is not loaded, load the segment tree
     if (segmentIndexMap == null) {
       // List<FileStatus> fileStatusList = new LinkedList<FileStatus>();
-      List<TableBlockInfo> tableBlockInfoList = new LinkedList<TableBlockInfo>();
+      List<TableBlockInfo> tableBlockInfoList =
+          getTableBlockInfo(job, absoluteTableIdentifier, segmentId);
       // getFileStatusOfSegments(job, new int[]{ segmentId }, fileStatusList);
 
-      // get file location of all files of given segment
-      JobContext newJob =
-          new JobContextImpl(new Configuration(job.getConfiguration()), job.getJobID());
-      newJob.getConfiguration().set(CarbonInputFormat.INPUT_SEGMENT_NUMBERS, segmentId + "");
-
-      // identify table blocks
-      for (InputSplit inputSplit : getSplitsInternal(newJob)) {
-        CarbonInputSplit carbonInputSplit = (CarbonInputSplit) inputSplit;
-        BlockletInfos blockletInfos = new BlockletInfos(carbonInputSplit.getNumberOfBlocklets(), 0,
-            carbonInputSplit.getNumberOfBlocklets());
-        tableBlockInfoList.add(
-            new TableBlockInfo(carbonInputSplit.getPath().toString(), carbonInputSplit.getStart(),
-                segmentId, carbonInputSplit.getLocations(), carbonInputSplit.getLength(),
-                blockletInfos));
-      }
-
       Map<String, List<TableBlockInfo>> segmentToTableBlocksInfos = new HashMap<>();
       segmentToTableBlocksInfos.put(segmentId, tableBlockInfoList);
 
@@ -621,7 +640,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
 
   @Override protected List<FileStatus> listStatus(JobContext job) throws IOException {
     List<FileStatus> result = new ArrayList<FileStatus>();
-    String[] segmentsToConsider = getValidSegments(job);
+    String[] segmentsToConsider = getSegmentsFromConfiguration(job);
     if (segmentsToConsider.length == 0) {
       throw new IOException("No segments found");
     }
@@ -703,7 +722,8 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
   /**
    * @return updateExtension
    */
-  private String[] getValidSegments(JobContext job) throws IOException {
+  private String[] getSegmentsFromConfiguration(JobContext job)
+      throws IOException {
     String segmentString = job.getConfiguration().get(INPUT_SEGMENT_NUMBERS, "");
     // if no segments
     if (segmentString.trim().isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 497d9f8..5ee3e83 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -23,6 +23,7 @@ import scala.collection.JavaConverters._
 import scala.reflect.ClassTag
 
 import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.InputSplit
 import org.apache.hadoop.mapreduce.Job
 import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
@@ -32,9 +33,11 @@ import org.apache.carbondata.common.CarbonIterator
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.cache.dictionary.Dictionary
 import org.apache.carbondata.core.carbon.datastore.block.{BlockletInfos, TableBlockInfo}
-import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
+import org.apache.carbondata.core.carbon.datastore.SegmentTaskIndexStore
+import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants}
 import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory
 import org.apache.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
+import org.apache.carbondata.lcm.status.SegmentStatusManager
 import org.apache.carbondata.scan.executor.QueryExecutorFactory
 import org.apache.carbondata.scan.expression.Expression
 import org.apache.carbondata.scan.model.QueryModel
@@ -82,15 +85,24 @@ class CarbonScanRDD[V: ClassTag](
 
     val result = new util.ArrayList[Partition](defaultParallelism)
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+    val validAndInvalidSegments = new SegmentStatusManager(queryModel.getAbsoluteTableIdentifier)
+        .getValidAndInvalidSegments
     // set filter resolver tree
     try {
       // before applying filter check whether segments are available in the table.
-      val splits = carbonInputFormat.getSplits(job)
-      if (!splits.isEmpty) {
+      if (!validAndInvalidSegments.getValidSegments.isEmpty) {
         val filterResolver = carbonInputFormat
           .getResolvedFilter(job.getConfiguration, filterExpression)
         CarbonInputFormat.setFilterPredicates(job.getConfiguration, filterResolver)
         queryModel.setFilterExpressionResolverTree(filterResolver)
+        CarbonInputFormat
+          .setSegmentsToAccess(job.getConfiguration,
+            validAndInvalidSegments.getValidSegments
+          )
+        SegmentTaskIndexStore.getInstance()
+          .removeTableBlocks(validAndInvalidSegments.getInvalidSegments,
+            queryModel.getAbsoluteTableIdentifier
+          )
       }
     }
     catch {
@@ -102,7 +114,7 @@ class CarbonScanRDD[V: ClassTag](
     val splits = carbonInputFormat.getSplits(job)
     if (!splits.isEmpty) {
       val carbonInputSplits = splits.asScala.map(_.asInstanceOf[CarbonInputSplit])
-
+      queryModel.setInvalidSegmentIds(validAndInvalidSegments.getInvalidSegments)
       val blockListTemp = carbonInputSplits.map(inputSplit =>
         new TableBlockInfo(inputSplit.getPath.toString,
           inputSplit.getStart, inputSplit.getSegmentId,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala
index ae29650..d9e1349 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/CompactionSystemLockFeatureTest.scala
@@ -118,7 +118,7 @@ class CompactionSystemLockFeatureTest extends QueryTest with BeforeAndAfterAll {
         )
     )
     // merged segment should not be there
-    val segments = segmentStatusManager.getValidSegments.listOfValidSegments.asScala.toList
+    val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
     assert(segments.contains("0.1"))
     assert(!segments.contains("0"))
     assert(!segments.contains("1"))
@@ -130,7 +130,7 @@ class CompactionSystemLockFeatureTest extends QueryTest with BeforeAndAfterAll {
         )
     )
     // merged segment should not be there
-    val segments2 = segmentStatusManager2.getValidSegments.listOfValidSegments.asScala.toList
+    val segments2 = segmentStatusManager2.getValidAndInvalidSegments.getValidSegments.asScala.toList
     assert(segments2.contains("0.1"))
     assert(!segments2.contains("0"))
     assert(!segments2.contains("1"))

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
index 7bbee54..4ec00ac 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
@@ -92,7 +92,7 @@ class DataCompactionCardinalityBoundryTest extends QueryTest with BeforeAndAfter
             new CarbonTableIdentifier("default", "cardinalityTest", "1")
           )
       )
-      val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+      val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
 
       if (!segments.contains("0.1")) {
         // wait for 2 seconds for compaction to complete.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
index b01e66b..4eb873a 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
@@ -104,7 +104,7 @@ class DataCompactionLockTest extends QueryTest with BeforeAndAfterAll {
       val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
         absoluteTableIdentifier
       )
-      val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+      val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
 
       if (!segments.contains("0.1")) {
         // wait for 2 seconds for compaction to complete.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala
index 09fb427..17fc1e5 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala
@@ -44,7 +44,7 @@ class DataCompactionNoDictionaryTest extends QueryTest with BeforeAndAfterAll {
           new CarbonTableIdentifier(databaseName, tableName.toLowerCase , tableId)
         )
     )
-    val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+    val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
     segments
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala
index 7737745..39dba52 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala
@@ -90,7 +90,7 @@ class DataCompactionTest extends QueryTest with BeforeAndAfterAll {
             new CarbonTableIdentifier("default", "normalcompaction", "1")
           )
       )
-      val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+      val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
 
       if (!segments.contains("0.1")) {
         // wait for 2 seconds for compaction to complete.
@@ -138,7 +138,7 @@ class DataCompactionTest extends QueryTest with BeforeAndAfterAll {
         )
     )
     // merged segment should not be there
-    val segments   = segmentStatusManager.getValidSegments.listOfValidSegments.asScala.toList
+    val segments   = segmentStatusManager.getValidAndInvalidSegments().getValidSegments.asScala.toList
     assert(!segments.contains("0"))
     assert(!segments.contains("1"))
     assert(!segments.contains("2"))

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
index a062153..9fe178f 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionIgnoreInMinorTest.scala
@@ -103,7 +103,7 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
             new CarbonTableIdentifier("default", "ignoremajor", noOfRetries + "")
           )
       )
-      val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+      val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
       segments.foreach(seg =>
         System.out.println( "valid segment is =" + seg)
       )
@@ -135,7 +135,7 @@ class MajorCompactionIgnoreInMinorTest extends QueryTest with BeforeAndAfterAll
         )
     )
     // merged segment should not be there
-    val segments = segmentStatusManager.getValidSegments.listOfValidSegments.asScala.toList
+    val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
     assert(segments.contains("0.1"))
     assert(segments.contains("2.1"))
     assert(!segments.contains("2"))

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
index 2b0afe6..3e51002 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/datacompaction/MajorCompactionStopsAfterCompaction.scala
@@ -93,7 +93,7 @@ class MajorCompactionStopsAfterCompaction extends QueryTest with BeforeAndAfterA
             new CarbonTableIdentifier("default", "stopmajor", noOfRetries + "")
           )
       )
-      val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+      val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
       segments.foreach(seg =>
         System.out.println( "valid segment is =" + seg)
       )
@@ -125,7 +125,7 @@ class MajorCompactionStopsAfterCompaction extends QueryTest with BeforeAndAfterA
         )
     )
     // merged segment should not be there
-    val segments = segmentStatusManager.getValidSegments.listOfValidSegments.asScala.toList
+    val segments = segmentStatusManager.getValidAndInvalidSegments.getValidSegments.asScala.toList
     assert(segments.contains("0.1"))
     assert(!segments.contains("0.2"))
     assert(!segments.contains("0"))

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/57ac4a5c/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java b/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
index 2b83b11..25e81f8 100644
--- a/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
+++ b/processing/src/main/java/org/apache/carbondata/lcm/status/SegmentStatusManager.java
@@ -49,7 +49,6 @@ import org.apache.carbondata.lcm.locks.ICarbonLock;
 import org.apache.carbondata.lcm.locks.LockUsage;
 
 import com.google.gson.Gson;
-
 /**
  * Manages Load/Segment status
  */
@@ -64,17 +63,6 @@ public class SegmentStatusManager {
     this.absoluteTableIdentifier = absoluteTableIdentifier;
   }
 
-  public static class ValidSegmentsInfo {
-    public final List<String> listOfValidSegments;
-    public final List<String> listOfValidUpdatedSegments;
-
-    public ValidSegmentsInfo(List<String> listOfValidSegments,
-        List<String> listOfValidUpdatedSegments) {
-      this.listOfValidSegments = listOfValidSegments;
-      this.listOfValidUpdatedSegments = listOfValidUpdatedSegments;
-    }
-  }
-
   /**
    * This will return the lock object used to lock the table status file before updation.
    *
@@ -89,9 +77,9 @@ public class SegmentStatusManager {
    * This method will return last modified time of tablestatus file
    */
   public long getTableStatusLastModifiedTime() throws IOException {
-    String tableStatusPath = CarbonStorePath.getCarbonTablePath(
-        absoluteTableIdentifier.getStorePath(), absoluteTableIdentifier.getCarbonTableIdentifier())
-          .getTableStatusFilePath();
+    String tableStatusPath = CarbonStorePath
+        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            absoluteTableIdentifier.getCarbonTableIdentifier()).getTableStatusFilePath();
     if (!FileFactory.isFileExist(tableStatusPath, FileFactory.getFileType(tableStatusPath))) {
       return 0L;
     } else {
@@ -102,14 +90,16 @@ public class SegmentStatusManager {
 
   /**
    * get valid segment for given table
+   *
    * @return
    * @throws IOException
    */
-  public ValidSegmentsInfo getValidSegments() throws IOException {
+  public ValidAndInvalidSegmentsInfo getValidAndInvalidSegments() throws IOException {
 
     // @TODO: move reading LoadStatus file to separate class
     List<String> listOfValidSegments = new ArrayList<String>(10);
     List<String> listOfValidUpdatedSegments = new ArrayList<String>(10);
+    List<String> listOfInvalidSegments = new ArrayList<String>(10);
     CarbonTablePath carbonTablePath = CarbonStorePath
         .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
             absoluteTableIdentifier.getCarbonTableIdentifier());
@@ -157,8 +147,15 @@ public class SegmentStatusManager {
               listOfValidUpdatedSegments.add(loadMetadataDetails.getLoadName());
             }
             listOfValidSegments.add(loadMetadataDetails.getLoadName());
-
+          } else if ((CarbonCommonConstants.STORE_LOADSTATUS_FAILURE
+              .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())
+              || CarbonCommonConstants.SEGMENT_COMPACTED
+              .equalsIgnoreCase(loadMetadataDetails.getLoadStatus())
+              || CarbonCommonConstants.MARKED_FOR_DELETE
+              .equalsIgnoreCase(loadMetadataDetails.getLoadStatus()))) {
+            listOfInvalidSegments.add(loadMetadataDetails.getLoadName());
           }
+
         }
       }
     } catch (IOException e) {
@@ -176,7 +173,8 @@ public class SegmentStatusManager {
       }
 
     }
-    return new ValidSegmentsInfo(listOfValidSegments, listOfValidUpdatedSegments);
+    return new ValidAndInvalidSegmentsInfo(listOfValidSegments, listOfValidUpdatedSegments,
+        listOfInvalidSegments);
   }
 
   /**
@@ -218,6 +216,7 @@ public class SegmentStatusManager {
 
   /**
    * returns current time
+   *
    * @return
    */
   private String readCurrentTime() {
@@ -244,6 +243,7 @@ public class SegmentStatusManager {
 
   /**
    * updates deletion status
+   *
    * @param loadIds
    * @param tableFolderPath
    * @return
@@ -271,13 +271,10 @@ public class SegmentStatusManager {
         listOfLoadFolderDetailsArray = readLoadMetadata(tableFolderPath);
         if (listOfLoadFolderDetailsArray != null && listOfLoadFolderDetailsArray.length != 0) {
           updateDeletionStatus(loadIds, listOfLoadFolderDetailsArray, invalidLoadIds);
-          if(invalidLoadIds.isEmpty())
-          {
+          if (invalidLoadIds.isEmpty()) {
             // All or None , if anything fails then dont write
             writeLoadDetailsIntoFile(dataLoadLocation, listOfLoadFolderDetailsArray);
-          }
-          else
-          {
+          } else {
             return invalidLoadIds;
           }
 
@@ -330,13 +327,11 @@ public class SegmentStatusManager {
         // read existing metadata details in load metadata.
         listOfLoadFolderDetailsArray = readLoadMetadata(tableFolderPath);
         if (listOfLoadFolderDetailsArray != null && listOfLoadFolderDetailsArray.length != 0) {
-          updateDeletionStatus(loadDate, listOfLoadFolderDetailsArray,
-              invalidLoadTimestamps, loadStartTime);
-          if(invalidLoadTimestamps.isEmpty()) {
+          updateDeletionStatus(loadDate, listOfLoadFolderDetailsArray, invalidLoadTimestamps,
+              loadStartTime);
+          if (invalidLoadTimestamps.isEmpty()) {
             writeLoadDetailsIntoFile(dataLoadLocation, listOfLoadFolderDetailsArray);
-          }
-          else
-          {
+          } else {
             return invalidLoadTimestamps;
           }
 
@@ -395,6 +390,7 @@ public class SegmentStatusManager {
 
   /**
    * updates deletion status details for each load and returns invalidLoadIds
+   *
    * @param loadIds
    * @param listOfLoadFolderDetailsArray
    * @param invalidLoadIds
@@ -503,17 +499,39 @@ public class SegmentStatusManager {
 
   /**
    * unlocks given file
+   *
    * @param carbonLock
    */
   private void fileUnlock(ICarbonLock carbonLock) {
     if (carbonLock.unlock()) {
       LOG.info("Metadata lock has been successfully released");
     } else {
-      LOG
-          .error("Not able to release the metadata lock");
+      LOG.error("Not able to release the metadata lock");
     }
   }
 
+  public static class ValidAndInvalidSegmentsInfo {
+    private final List<String> listOfValidSegments;
+    private final List<String> listOfValidUpdatedSegments;
+    private final List<String> listOfInvalidSegments;
+
+    private ValidAndInvalidSegmentsInfo(List<String> listOfValidSegments,
+        List<String> listOfValidUpdatedSegments, List<String> listOfInvalidUpdatedSegments) {
+      this.listOfValidSegments = listOfValidSegments;
+      this.listOfValidUpdatedSegments = listOfValidUpdatedSegments;
+      this.listOfInvalidSegments = listOfInvalidUpdatedSegments;
+    }
+
+    public List<String> getInvalidSegments() {
+      return listOfInvalidSegments;
+    }
 
+    public List<String> getValidSegments() {
+      return listOfValidSegments;
+    }
 
+    public List<String> getUpadtedSegments() {
+      return listOfValidUpdatedSegments;
+    }
+  }
 }



[29/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-253] MemoryLeak during blocklet distribution at executor btree as btree is loaded every time for same block. This closes #170

Posted by ra...@apache.org.
[CARBONDATA-253] MemoryLeak during blocklet distribution at executor btree as btree is loaded every time for same block. This closes #170


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

Branch: refs/heads/branch-0.1
Commit: 8b6429a8e503acafb06c60883d43150270486dc3
Parents: 303ccd9
Author: Venkata Ramana G <ra...@huawei.com>
Authored: Sun Sep 18 04:11:05 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 10:01:44 2016 +0530

----------------------------------------------------------------------

----------------------------------------------------------------------



[16/50] [abbrv] incubator-carbondata git commit: [Issue Number] CARBONDATA-242 [Description] When user provides Null member inside NOT IN filter condition the resultset is not compatible with hive result

Posted by ra...@apache.org.
[Issue Number] CARBONDATA-242
[Description] When user provides Null member inside NOT IN filter condition the resultset is not compatible with hive result


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

Branch: refs/heads/branch-0.1
Commit: 772382dfd869581b09cea75513e3827818080b7b
Parents: 145c7af
Author: sujith71955 <su...@gmail.com>
Authored: Thu Sep 15 14:53:04 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:40:57 2016 +0530

----------------------------------------------------------------------
 .../expression/logical/FalseExpression.java     | 51 ++++++++++++++++++++
 .../scan/filter/FilterExpressionProcessor.java  |  6 ++-
 .../scan/filter/intf/ExpressionType.java        |  3 +-
 .../apache/carbondata/spark/CarbonFilters.scala | 35 +++++++++++---
 .../GrtLtFilterProcessorTestCase.scala          |  6 +++
 5 files changed, 93 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/772382df/core/src/main/java/org/apache/carbondata/scan/expression/logical/FalseExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/expression/logical/FalseExpression.java b/core/src/main/java/org/apache/carbondata/scan/expression/logical/FalseExpression.java
new file mode 100644
index 0000000..2b0aead
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/scan/expression/logical/FalseExpression.java
@@ -0,0 +1,51 @@
+package org.apache.carbondata.scan.expression.logical;
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.scan.expression.Expression;
+import org.apache.carbondata.scan.expression.ExpressionResult;
+import org.apache.carbondata.scan.expression.LiteralExpression;
+import org.apache.carbondata.scan.expression.conditional.BinaryConditionalExpression;
+import org.apache.carbondata.scan.expression.exception.FilterIllegalMemberException;
+import org.apache.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.apache.carbondata.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.scan.filter.intf.RowIntf;
+
+
+
+/**
+ * This class will form an expression whose evaluation will be always false.
+ */
+public class FalseExpression  extends BinaryConditionalExpression {
+
+
+  private static final long serialVersionUID = -8390184061336799370L;
+
+  public FalseExpression(Expression child1) {
+    super(child1, new LiteralExpression(null,null));
+  }
+
+  /**
+   * This method will always return false, mainly used in the filter expressions
+   * which are illogical.
+   * eg: columnName NOT IN('Java',NULL)
+   * @param value
+   * @return
+   * @throws FilterUnsupportedException
+   * @throws FilterIllegalMemberException
+   */
+  @Override public ExpressionResult evaluate(RowIntf value)
+      throws FilterUnsupportedException, FilterIllegalMemberException {
+    return new ExpressionResult(DataType.BOOLEAN,false);
+  }
+
+  /**
+   * This method will return the expression types
+   * @return
+   */
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.FALSE;
+  }
+  @Override public String getString() {
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/772382df/core/src/main/java/org/apache/carbondata/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/apache/carbondata/scan/filter/FilterExpressionProcessor.java
index 9d996a0..b50e6e6 100644
--- a/core/src/main/java/org/apache/carbondata/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/apache/carbondata/scan/filter/FilterExpressionProcessor.java
@@ -245,7 +245,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
       case NOT_IN:
         return getFilterResolverBasedOnExpressionType(ExpressionType.NOT_EQUALS, false,
             expressionTree, tableIdentifier, expressionTree);
-
+      case FALSE:
+        return getFilterResolverBasedOnExpressionType(ExpressionType.FALSE, false,
+            expressionTree, tableIdentifier, expressionTree);
       default:
         return getFilterResolverBasedOnExpressionType(ExpressionType.UNKNOWN, false, expressionTree,
             tableIdentifier, expressionTree);
@@ -262,6 +264,8 @@ public class FilterExpressionProcessor implements FilterProcessor {
     BinaryConditionalExpression currentCondExpression = null;
     ConditionalExpression condExpression = null;
     switch (filterExpressionType) {
+      case FALSE:
+        return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
       case EQUALS:
         currentCondExpression = (BinaryConditionalExpression) expression;
         if (currentCondExpression.isSingleDimension()

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/772382df/core/src/main/java/org/apache/carbondata/scan/filter/intf/ExpressionType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/filter/intf/ExpressionType.java b/core/src/main/java/org/apache/carbondata/scan/filter/intf/ExpressionType.java
index 4d658fc..0dca5a4 100644
--- a/core/src/main/java/org/apache/carbondata/scan/filter/intf/ExpressionType.java
+++ b/core/src/main/java/org/apache/carbondata/scan/filter/intf/ExpressionType.java
@@ -39,6 +39,7 @@ public enum ExpressionType {
   NOT_IN,
   UNKNOWN,
   LITERAL,
-  RANGE
+  RANGE,
+  FALSE
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/772382df/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
index d16d583..5fb0051 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/CarbonFilters.scala
@@ -29,7 +29,7 @@ import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn
 import org.apache.carbondata.scan.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
 import org.apache.carbondata.scan.expression.conditional._
-import org.apache.carbondata.scan.expression.logical.{AndExpression, OrExpression}
+import org.apache.carbondata.scan.expression.logical.{AndExpression, FalseExpression, OrExpression}
 import org.apache.carbondata.spark.util.CarbonScalaUtil
 
 /**
@@ -258,16 +258,30 @@ object CarbonFilters {
         case Not(EqualTo(l@Literal(v, t), Cast(a: Attribute, _))) => new
             Some(new NotEqualsExpression(transformExpression(a).get, transformExpression(l).get))
 
-        case Not(In(a: Attribute, list)) if !list.exists(!_.isInstanceOf[Literal]) =>
+        case Not(In(a: Attribute, list))
+         if !list.exists(!_.isInstanceOf[Literal]) =>
+         if (list.exists(x => (isNullLiteral(x.asInstanceOf[Literal])))) {
+          Some(new FalseExpression(transformExpression(a).get))
+         }
+        else {
           Some(new NotInExpression(transformExpression(a).get,
-            new ListExpression(convertToJavaList(list.map(transformExpression(_).get)))))
+              new ListExpression(convertToJavaList(list.map(transformExpression(_).get)))))
+            }
         case In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) =>
           Some(new InExpression(transformExpression(a).get,
             new ListExpression(convertToJavaList(list.map(transformExpression(_).get)))))
         case Not(In(Cast(a: Attribute, _), list))
           if !list.exists(!_.isInstanceOf[Literal]) =>
-          Some(new NotInExpression(transformExpression(a).get,
-            new ListExpression(convertToJavaList(list.map(transformExpression(_).get)))))
+        /* if any illogical expression comes in NOT IN Filter like
+         NOT IN('scala',NULL) this will be treated as false expression and will
+         always return no result. */
+          if (list.exists(x => (isNullLiteral(x.asInstanceOf[Literal])))) {
+          Some(new FalseExpression(transformExpression(a).get))
+         }
+        else {
+          Some(new NotInExpression(transformExpression(a).get, new ListExpression(
+              convertToJavaList(list.map(transformExpression(_).get)))))
+              }
         case In(Cast(a: Attribute, _), list) if !list.exists(!_.isInstanceOf[Literal]) =>
           Some(new InExpression(transformExpression(a).get,
             new ListExpression(convertToJavaList(list.map(transformExpression(_).get)))))
@@ -335,7 +349,16 @@ object CarbonFilters {
     }
     exprs.flatMap(transformExpression(_, false)).reduceOption(new AndExpression(_, _))
   }
-
+  private def isNullLiteral(exp: Expression): Boolean = {
+    if (null != exp
+        &&  exp.isInstanceOf[Literal]
+        && (exp.asInstanceOf[Literal].dataType == org.apache.spark.sql.types.DataTypes.NullType)
+        || (exp.asInstanceOf[Literal].value == null)) {
+      true
+    } else {
+      false
+    }
+  }
   private def getActualCarbonDataType(column: String, carbonTable: CarbonTable) = {
     var carbonColumn: CarbonColumn =
       carbonTable.getDimensionByName(carbonTable.getFactTableName, column)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/772382df/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
index c2a4b80..5278344 100644
--- a/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
+++ b/integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
@@ -136,6 +136,12 @@ class GrtLtFilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
       Seq(Row(0))
     )
   }
+      test("In condition With improper format query regarding Null filter") {
+    checkAnswer(
+      sql("select empid from a12_allnull " + "where empid not in ('china',NULL)"),
+      Seq()
+    )
+      }
 
   //no null test cases
 


[06/50] [abbrv] incubator-carbondata git commit: add more query statistics

Posted by ra...@apache.org.
add more query statistics

example recovery

show statistics as table

add query id for decoder

add common constants file

print driver log and executor log seperately

add single recorder

and lock for log

fixed example


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

Branch: refs/heads/branch-0.1
Commit: 346a869c1b4eac21e78719c976b30fc0f4f45896
Parents: ce34e10
Author: foryou2030 <fo...@126.com>
Authored: Wed Aug 24 19:20:07 2016 +0800
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Sep 22 09:31:05 2016 +0530

----------------------------------------------------------------------
 .../DriverQueryStatisticsRecorder.java          | 183 +++++++++++++++++++
 .../carbon/querystatistics/QueryStatistic.java  |  24 +++
 .../QueryStatisticsConstants.java               |  53 ++++++
 .../QueryStatisticsRecorder.java                |  90 +++++++++
 .../core/util/CarbonTimeStatisticsFactory.java  |  13 +-
 .../apache/carbondata/core/util/CarbonUtil.java |  15 ++
 .../executor/impl/AbstractQueryExecutor.java    |  11 +-
 .../AbstractDetailQueryResultIterator.java      |  30 ++-
 .../iterator/DetailQueryResultIterator.java     |   2 +
 .../carbondata/hadoop/CarbonInputFormat.java    |   3 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  49 +++--
 .../carbondata/spark/util/CarbonScalaUtil.scala |   1 +
 .../org/apache/spark/sql/CarbonContext.scala    |  15 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  26 ++-
 .../org/apache/spark/sql/CarbonOperators.scala  |   2 +-
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |  12 +-
 16 files changed, 491 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
new file mode 100644
index 0000000..9e83c22
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/DriverQueryStatisticsRecorder.java
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.querystatistics;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.logging.LogService;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+
+import static org.apache.carbondata.core.util.CarbonUtil.printLine;
+
+/**
+ * Class will be used to record and log the query statistics
+ */
+public class DriverQueryStatisticsRecorder {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DriverQueryStatisticsRecorder.class.getName());
+
+  /**
+   * singleton QueryStatisticsRecorder for driver
+   */
+  private Map<String, List<QueryStatistic>> queryStatisticsMap;
+
+  /**
+   * lock for log statistics table
+   */
+  private static final Object lock = new Object();
+
+  private DriverQueryStatisticsRecorder() {
+    queryStatisticsMap = new HashMap<String, List<QueryStatistic>>();
+  }
+
+  private static DriverQueryStatisticsRecorder carbonLoadStatisticsImplInstance =
+      new DriverQueryStatisticsRecorder();
+
+  public static DriverQueryStatisticsRecorder getInstance() {
+    return carbonLoadStatisticsImplInstance;
+  }
+
+  /**
+   * Below method will be used to add the statistics
+   *
+   * @param statistic
+   */
+  public synchronized void recordStatisticsForDriver(QueryStatistic statistic, String queryId) {
+    // refresh query Statistics Map
+    if (queryStatisticsMap.get(queryId) != null) {
+      queryStatisticsMap.get(queryId).add(statistic);
+    } else {
+      List<QueryStatistic> newQueryStatistics = new ArrayList<QueryStatistic>();
+      newQueryStatistics.add(statistic);
+      queryStatisticsMap.put(queryId, newQueryStatistics);
+    }
+  }
+
+  /**
+   * Below method will be used to show statistic log as table
+   */
+  public void logStatisticsAsTableDriver() {
+    synchronized (lock) {
+      String tableInfo = collectDriverStatistics();
+      if (null != tableInfo) {
+        LOGGER.statistic(tableInfo);
+      }
+    }
+  }
+
+  /**
+   * Below method will parse queryStatisticsMap and put time into table
+   */
+  public String collectDriverStatistics() {
+    for (String key: queryStatisticsMap.keySet()) {
+      try {
+        // TODO: get the finished query, and print Statistics
+        if (queryStatisticsMap.get(key).size() > 3) {
+          String sql_parse_time = "";
+          String load_meta_time = "";
+          String block_allocation_time = "";
+          String block_identification_time = "";
+          Double driver_part_time_tmp = 0.0;
+          String splitChar = " ";
+          // get statistic time from the QueryStatistic
+          for (QueryStatistic statistic : queryStatisticsMap.get(key)) {
+            switch (statistic.getMessage()) {
+              case QueryStatisticsConstants.SQL_PARSE:
+                sql_parse_time += statistic.getTimeTaken() + splitChar;
+                driver_part_time_tmp += statistic.getTimeTaken();
+                break;
+              case QueryStatisticsConstants.LOAD_META:
+                load_meta_time += statistic.getTimeTaken() + splitChar;
+                driver_part_time_tmp += statistic.getTimeTaken();
+                break;
+              case QueryStatisticsConstants.BLOCK_ALLOCATION:
+                block_allocation_time += statistic.getTimeTaken() + splitChar;
+                driver_part_time_tmp += statistic.getTimeTaken();
+                break;
+              case QueryStatisticsConstants.BLOCK_IDENTIFICATION:
+                block_identification_time += statistic.getTimeTaken() + splitChar;
+                driver_part_time_tmp += statistic.getTimeTaken();
+                break;
+              default:
+                break;
+            }
+          }
+          String driver_part_time = driver_part_time_tmp + splitChar;
+          // structure the query statistics info table
+          StringBuilder tableInfo = new StringBuilder();
+          int len1 = 8;
+          int len2 = 20;
+          int len3 = 21;
+          int len4 = 22;
+          String line = "+" + printLine("-", len1) + "+" + printLine("-", len2) + "+" +
+              printLine("-", len3) + "+" + printLine("-", len4) + "+";
+          String line2 = "|" + printLine(" ", len1) + "+" + printLine("-", len2) + "+" +
+              printLine(" ", len3) + "+" + printLine("-", len4) + "+";
+          // table header
+          tableInfo.append(line).append("\n");
+          tableInfo.append("|" + printLine(" ", (len1 - "Module".length())) + "Module" + "|" +
+              printLine(" ", (len2 - "Operation Step".length())) + "Operation Step" + "|" +
+              printLine(" ", (len3 + len4 + 1 - "Query Cost".length())) +
+              "Query Cost" + "|" + "\n");
+          // driver part
+          tableInfo.append(line).append("\n");
+          tableInfo.append("|" + printLine(" ", len1) + "|" +
+              printLine(" ", (len2 - "SQL parse".length())) + "SQL parse" + "|" +
+              printLine(" ", len3) + "|" +
+              printLine(" ", (len4 - sql_parse_time.length())) + sql_parse_time + "|" + "\n");
+          tableInfo.append(line2).append("\n");
+          tableInfo.append("|" +printLine(" ", (len1 - "Driver".length())) + "Driver" + "|" +
+              printLine(" ", (len2 - "Load meta data".length())) + "Load meta data" + "|" +
+              printLine(" ", (len3 - driver_part_time.length())) + driver_part_time + "|" +
+              printLine(" ", (len4 - load_meta_time.length())) +
+              load_meta_time + "|" + "\n");
+          tableInfo.append(line2).append("\n");
+          tableInfo.append("|" +
+              printLine(" ", (len1 - "Part".length())) + "Part" + "|" +
+              printLine(" ", (len2 - "Block allocation".length())) +
+              "Block allocation" + "|" +
+              printLine(" ", len3) + "|" +
+              printLine(" ", (len4 - block_allocation_time.length())) +
+              block_allocation_time + "|" + "\n");
+          tableInfo.append(line2).append("\n");
+          tableInfo.append("|" +
+              printLine(" ", len1) + "|" +
+              printLine(" ", (len2 - "Block identification".length())) +
+              "Block identification" + "|" +
+              printLine(" ", len3) + "|" +
+              printLine(" ", (len4 - block_identification_time.length())) +
+              block_identification_time + "|" + "\n");
+          tableInfo.append(line).append("\n");
+
+          // once the statistics be printed, remove it from the map
+          queryStatisticsMap.remove(key);
+          // show query statistic as "query id" + "table"
+          return "Print query statistic for query id: " + key + "\n" + tableInfo.toString();
+        }
+      } catch (Exception ex) {
+        return "Put statistics into table failed, catch exception: " + ex.getMessage();
+      }
+    }
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java
index 39198c8..3e44949 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatistic.java
@@ -45,6 +45,11 @@ public class QueryStatistic implements Serializable {
    */
   private long startTime;
 
+  /**
+   * number of count
+   */
+  private long count;
+
   public QueryStatistic() {
     this.startTime = System.currentTimeMillis();
   }
@@ -72,6 +77,12 @@ public class QueryStatistic implements Serializable {
     this.message = message;
   }
 
+  public void addCountStatistic(String message, long count) {
+    this.timeTaken = -1;
+    this.count = count;
+    this.message = message;
+  }
+
   /**
    * Below method will be used to get the statistic message, which will
    * be used to log
@@ -82,4 +93,17 @@ public class QueryStatistic implements Serializable {
   public String getStatistics(String queryWithTaskId) {
     return message + " for the taskid : " + queryWithTaskId + " Is : " + timeTaken;
   }
+
+  public String getMessage() {
+    return this.message;
+  }
+
+  public double getTimeTaken() {
+    return (double)this.timeTaken/1000;
+  }
+
+  public long getCount() {
+    return this.count;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java
new file mode 100644
index 0000000..540cf1e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsConstants.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.core.carbon.querystatistics;
+
+public interface QueryStatisticsConstants {
+
+  // driver side
+  String SQL_PARSE = "Time taken to parse sql In Driver Side";
+
+  String LOAD_META = "Time taken to load meta data In Driver Side";
+
+  String LOAD_BLOCKS_DRIVER = "Time taken to load the Block(s) In Driver Side";
+
+  String BLOCK_ALLOCATION = "Total Time taken in block(s) allocation";
+
+  String BLOCK_IDENTIFICATION = "Time taken to identify Block(s) to scan";
+
+  // executor side
+  String EXECUTOR_PART =
+      "Total Time taken to execute the query in executor Side";
+
+  String LOAD_BLOCKS_EXECUTOR = "Time taken to load the Block(s) In Executor";
+
+  String SCAN_BLOCKS_NUM = "The num of blocks scanned";
+
+  String SCAN_BLOCKS_TIME = "Time taken to scan blocks";
+
+  String LOAD_DICTIONARY = "Time taken to load the Dictionary In Executor";
+
+  String PREPARE_RESULT = "Total Time taken to prepare query result";
+
+  String RESULT_SIZE = "The size of query result";
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java
index ce12cae..e713237 100644
--- a/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/querystatistics/QueryStatisticsRecorder.java
@@ -25,6 +25,8 @@ import java.util.List;
 import org.apache.carbondata.common.logging.LogService;
 import org.apache.carbondata.common.logging.LogServiceFactory;
 
+import static org.apache.carbondata.core.util.CarbonUtil.printLine;
+
 /**
  * Class will be used to record and log the query statistics
  */
@@ -32,6 +34,7 @@ public class QueryStatisticsRecorder implements Serializable {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(QueryStatisticsRecorder.class.getName());
+
   /**
    * serialization version
    */
@@ -49,6 +52,11 @@ public class QueryStatisticsRecorder implements Serializable {
    */
   private String queryIWthTask;
 
+  /**
+   * lock for log statistics table
+   */
+  private static final Object lock = new Object();
+
   public QueryStatisticsRecorder(String queryId) {
     queryStatistics = new ArrayList<QueryStatistic>();
     this.queryIWthTask = queryId;
@@ -71,4 +79,86 @@ public class QueryStatisticsRecorder implements Serializable {
       LOGGER.statistic(statistic.getStatistics(queryIWthTask));
     }
   }
+
+  /**
+   * Below method will be used to show statistic log as table
+   */
+  public void logStatisticsAsTableExecutor() {
+    synchronized (lock) {
+      String tableInfo = collectExecutorStatistics();
+      if (null != tableInfo) {
+        LOGGER.statistic(tableInfo);
+      }
+    }
+  }
+
+  /**
+   * Below method will parse queryStatisticsMap and put time into table
+   */
+  public String collectExecutorStatistics() {
+    String load_blocks_time = "";
+    String scan_blocks_time = "";
+    String scan_blocks_num = "";
+    String load_dictionary_time = "";
+    String result_size = "";
+    String total_executor_time = "";
+    String splitChar = " ";
+    try {
+      for (QueryStatistic statistic : queryStatistics) {
+        switch (statistic.getMessage()) {
+          case QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR:
+            load_blocks_time += statistic.getTimeTaken() + splitChar;
+            break;
+          case QueryStatisticsConstants.SCAN_BLOCKS_TIME:
+            scan_blocks_time += statistic.getTimeTaken() + splitChar;
+            break;
+          case QueryStatisticsConstants.SCAN_BLOCKS_NUM:
+            scan_blocks_num += statistic.getCount() + splitChar;
+            break;
+          case QueryStatisticsConstants.LOAD_DICTIONARY:
+            load_dictionary_time += statistic.getTimeTaken() + splitChar;
+            break;
+          case QueryStatisticsConstants.RESULT_SIZE:
+            result_size += statistic.getCount() + " ";
+            break;
+          case QueryStatisticsConstants.EXECUTOR_PART:
+            total_executor_time += statistic.getTimeTaken() + splitChar;
+            break;
+          default:
+            break;
+        }
+      }
+      String headers = "task_id,load_blocks_time,load_dictionary_time,scan_blocks_time," +
+          "scan_blocks_num,result_size,total_executor_time";
+      List<String> values = new ArrayList<String>();
+      values.add(queryIWthTask);
+      values.add(load_blocks_time);
+      values.add(load_dictionary_time);
+      values.add(scan_blocks_time);
+      values.add(scan_blocks_num);
+      values.add(result_size);
+      values.add(total_executor_time);
+      StringBuilder tableInfo = new StringBuilder();
+      String[] columns = headers.split(",");
+      String line = "";
+      String hearLine = "";
+      String valueLine = "";
+      for (int i = 0; i < columns.length; i++) {
+        int len = Math.max(columns[i].length(), values.get(i).length());
+        line += "+" + printLine("-", len);
+        hearLine += "|" + printLine(" ", len - columns[i].length()) + columns[i];
+        valueLine += "|" + printLine(" ", len - values.get(i).length()) + values.get(i);
+      }
+      // struct table info
+      tableInfo.append(line + "+").append("\n");
+      tableInfo.append(hearLine + "|").append("\n");
+      tableInfo.append(line + "+").append("\n");
+      tableInfo.append(valueLine + "|").append("\n");
+      tableInfo.append(line + "+").append("\n");
+      return "Print query statistic for each task id:" + "\n" + tableInfo.toString();
+    } catch (Exception ex) {
+      return "Put statistics into table failed, catch exception: " + ex.getMessage();
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/core/src/main/java/org/apache/carbondata/core/util/CarbonTimeStatisticsFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonTimeStatisticsFactory.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonTimeStatisticsFactory.java
index c7c2b8a..274800f 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonTimeStatisticsFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonTimeStatisticsFactory.java
@@ -17,15 +17,18 @@
 
 package org.apache.carbondata.core.util;
 
+import org.apache.carbondata.core.carbon.querystatistics.DriverQueryStatisticsRecorder;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 
 public class CarbonTimeStatisticsFactory {
   private static String LoadStatisticsInstanceType;
   private static LoadStatistics LoadStatisticsInstance;
+  private static DriverQueryStatisticsRecorder QueryStatisticsRecorderInstance;
 
   static {
     CarbonTimeStatisticsFactory.updateTimeStatisticsUtilStatus();
     LoadStatisticsInstance = genLoadStatisticsInstance();
+    QueryStatisticsRecorderInstance = genQueryStatisticsRecorderInstance();
   }
 
   private static void updateTimeStatisticsUtilStatus() {
@@ -49,4 +52,12 @@ public class CarbonTimeStatisticsFactory {
     return LoadStatisticsInstance;
   }
 
-}
+  private static DriverQueryStatisticsRecorder genQueryStatisticsRecorderInstance() {
+    return DriverQueryStatisticsRecorder.getInstance();
+  }
+
+  public static DriverQueryStatisticsRecorder getQueryStatisticsRecorderInstance() {
+    return QueryStatisticsRecorderInstance;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/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 df538e0..77d041c 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
@@ -1415,5 +1415,20 @@ public final class CarbonUtil {
     }
     return readLine;
   }
+
+  /**
+   * Below method will create string like "***********"
+   *
+   * @param a
+   * @param num
+   */
+  public static String printLine(String a, int num)
+  {
+    StringBuilder builder = new StringBuilder();
+    for (int i = 0; i < num; i++) {
+      builder.append(a);
+    }
+    return builder.toString();
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
index 832b2fa..dab8a23 100644
--- a/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/apache/carbondata/scan/executor/impl/AbstractQueryExecutor.java
@@ -38,6 +38,7 @@ import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatistic;
+import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsConstants;
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsRecorder;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
@@ -108,7 +109,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
       throw new QueryExecutionException(e);
     }
     queryStatistic
-        .addStatistics("Time taken to load the Block(s) In Executor", System.currentTimeMillis());
+        .addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_EXECUTOR, System.currentTimeMillis());
     queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
     //
     // // updating the restructuring infos for the query
@@ -150,7 +151,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
         .getDimensionDictionaryDetail(queryModel.getQueryDimension(),
             queryProperties.complexFilterDimension, queryModel.getAbsoluteTableIdentifier());
     queryStatistic
-        .addStatistics("Time taken to load the Dictionary In Executor", System.currentTimeMillis());
+        .addStatistics(QueryStatisticsConstants.LOAD_DICTIONARY, System.currentTimeMillis());
     queryProperties.queryStatisticsRecorder.recordStatistics(queryStatistic);
     queryModel.setColumnToDictionaryMapping(queryProperties.columnToDictionayMapping);
     // setting the sort dimension index. as it will be updated while getting the sort info
@@ -207,6 +208,12 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     }
     queryProperties.complexDimensionInfoMap =
         blockExecutionInfoList.get(blockExecutionInfoList.size() - 1).getComlexDimensionInfoMap();
+    if (null != queryModel.getStatisticsRecorder()) {
+      QueryStatistic queryStatistic = new QueryStatistic();
+      queryStatistic.addCountStatistic(QueryStatisticsConstants.SCAN_BLOCKS_NUM,
+          blockExecutionInfoList.size());
+      queryModel.getStatisticsRecorder().recordStatistics(queryStatistic);
+    }
     return blockExecutionInfoList;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/core/src/main/java/org/apache/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
index d583560..36c4ad0 100644
--- a/core/src/main/java/org/apache/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -26,6 +26,9 @@ import org.apache.carbondata.common.logging.LogServiceFactory;
 import org.apache.carbondata.core.carbon.datastore.DataRefNode;
 import org.apache.carbondata.core.carbon.datastore.DataRefNodeFinder;
 import org.apache.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
+import org.apache.carbondata.core.carbon.querystatistics.QueryStatistic;
+import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsConstants;
+import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsRecorder;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.datastorage.store.FileHolder;
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory;
@@ -67,6 +70,21 @@ public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {
 
   protected boolean nextBatch = false;
 
+  /**
+   * total time scan the blocks
+   */
+  protected long totalScanTime;
+
+  /**
+   * is the statistic recorded
+   */
+  protected boolean isStatisticsRecorded;
+
+  /**
+   *  QueryStatisticsRecorder
+   */
+  protected QueryStatisticsRecorder recorder;
+
   public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
     String batchSizeString =
         CarbonProperties.getInstance().getProperty(CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE);
@@ -80,7 +98,7 @@ public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {
     } else {
       batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
     }
-
+    this.recorder = queryModel.getStatisticsRecorder();
     this.blockExecutionInfos = infos;
     this.fileReader = FileFactory.getFileHolder(
         FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getStorePath()));
@@ -111,8 +129,16 @@ public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {
   @Override public boolean hasNext() {
     if ((dataBlockIterator != null && dataBlockIterator.hasNext()) || nextBatch) {
       return true;
+    } else if (blockExecutionInfos.size() > 0) {
+      return true;
     } else {
-      return blockExecutionInfos.size() > 0;
+      if (!isStatisticsRecorded) {
+        QueryStatistic statistic = new QueryStatistic();
+        statistic.addFixedTimeStatistic(QueryStatisticsConstants.SCAN_BLOCKS_TIME, totalScanTime);
+        recorder.recordStatistics(statistic);
+        isStatisticsRecorded = true;
+      }
+      return false;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java
index fa804a5..0013c0a 100644
--- a/core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/scan/result/iterator/DetailQueryResultIterator.java
@@ -49,6 +49,7 @@ public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator
 
   @Override public BatchResult next() {
     BatchResult result;
+    long startTime = System.currentTimeMillis();
     try {
       if (future == null) {
         future = execute();
@@ -63,6 +64,7 @@ public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator
         execService.awaitTermination(1, TimeUnit.HOURS);
         fileReader.finish();
       }
+      totalScanTime += System.currentTimeMillis() - startTime;
     } catch (Exception ex) {
       execService.shutdown();
       fileReader.finish();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
index 6bc692f..da697ad 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputFormat.java
@@ -49,6 +49,7 @@ import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.carbon.path.CarbonStorePath;
 import org.apache.carbondata.core.carbon.path.CarbonTablePath;
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatistic;
+import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsConstants;
 import org.apache.carbondata.core.carbon.querystatistics.QueryStatisticsRecorder;
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.KeyGenException;
@@ -483,7 +484,7 @@ public class CarbonInputFormat<T> extends FileInputFormat<Void, T> {
       }
       resultFilterredBlocks.addAll(filterredBlocks);
     }
-    statistic.addStatistics("Time taken to load the Block(s) in Driver Side",
+    statistic.addStatistics(QueryStatisticsConstants.LOAD_BLOCKS_DRIVER,
         System.currentTimeMillis());
     recorder.recordStatistics(statistic);
     recorder.logStatistics();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
index 6c0438f..5f50644 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -32,7 +32,8 @@ import org.apache.carbondata.common.CarbonIterator
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.cache.dictionary.Dictionary
 import org.apache.carbondata.core.carbon.datastore.block.{BlockletInfos, TableBlockInfo}
-import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsRecorder}
+import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
+import org.apache.carbondata.core.util.CarbonTimeStatisticsFactory
 import org.apache.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
 import org.apache.carbondata.scan.executor.QueryExecutorFactory
 import org.apache.carbondata.scan.expression.Expression
@@ -75,8 +76,7 @@ class CarbonScanRDD[V: ClassTag](
   val defaultParallelism = sc.defaultParallelism
 
   override def getPartitions: Array[Partition] = {
-    val statisticRecorder = new QueryStatisticsRecorder(queryModel.getQueryId)
-    val startTime = System.currentTimeMillis()
+    val statisticRecorder = CarbonTimeStatisticsFactory.getQueryStatisticsRecorderInstance()
     val (carbonInputFormat: CarbonInputFormat[Array[Object]], job: Job) =
       QueryPlanUtil.createCarbonInputFormat(queryModel.getAbsoluteTableIdentifier)
 
@@ -113,19 +113,17 @@ class CarbonScanRDD[V: ClassTag](
       val blockList = CarbonLoaderUtil.
         distributeBlockLets(blockListTemp.asJava, defaultParallelism).asScala
       if (blockList.nonEmpty) {
+        var statistic = new QueryStatistic()
         // group blocks to nodes, tasks
-        val startTime = System.currentTimeMillis
-        var statistic = new QueryStatistic
         val activeNodes = DistributionUtil
           .ensureExecutorsAndGetNodeList(blockList.toArray, sparkContext)
         val nodeBlockMapping =
           CarbonLoaderUtil.nodeBlockTaskMapping(blockList.asJava, -1, defaultParallelism,
             activeNodes.toList.asJava
           )
-        val timeElapsed: Long = System.currentTimeMillis - startTime
-        statistic.addStatistics("Total Time taken in block(s) allocation", System.currentTimeMillis)
-        statisticRecorder.recordStatistics(statistic);
-        statistic = new QueryStatistic
+        statistic.addStatistics(QueryStatisticsConstants.BLOCK_ALLOCATION, System.currentTimeMillis)
+        statisticRecorder.recordStatisticsForDriver(statistic, queryModel.getQueryId())
+        statistic = new QueryStatistic()
         var i = 0
         // Create Spark Partition for each task and assign blocks
         nodeBlockMapping.asScala.foreach { entry =>
@@ -146,9 +144,10 @@ class CarbonScanRDD[V: ClassTag](
                 + s"parallelism: $defaultParallelism , " +
                 s"no.of.nodes: $noOfNodes, no.of.tasks: $noOfTasks"
         )
-        statistic.addStatistics("Time taken to identify Block(s) to scan", System.currentTimeMillis)
-        statisticRecorder.recordStatistics(statistic);
-        statisticRecorder.logStatistics
+        statistic.addStatistics(QueryStatisticsConstants.BLOCK_IDENTIFICATION,
+          System.currentTimeMillis)
+        statisticRecorder.recordStatisticsForDriver(statistic, queryModel.getQueryId())
+        statisticRecorder.logStatisticsAsTableDriver()
         result.asScala.foreach { r =>
           val cp = r.asInstanceOf[CarbonSparkPartition]
           logInfo(s"Node : " + cp.locations.toSeq.mkString(",")
@@ -216,13 +215,18 @@ class CarbonScanRDD[V: ClassTag](
         if (finished) {
           clearDictionaryCache(queryModel.getColumnToDictionaryMapping)
           if (null != queryModel.getStatisticsRecorder) {
-            val queryStatistic = new QueryStatistic
+            var queryStatistic = new QueryStatistic()
             queryStatistic
-              .addFixedTimeStatistic("Total Time taken to execute the query in executor Side",
+              .addFixedTimeStatistic(QueryStatisticsConstants.EXECUTOR_PART,
                 System.currentTimeMillis - queryStartTime
               )
-            queryModel.getStatisticsRecorder.recordStatistics(queryStatistic);
-            queryModel.getStatisticsRecorder.logStatistics();
+            queryModel.getStatisticsRecorder.recordStatistics(queryStatistic)
+            // result size
+            queryStatistic = new QueryStatistic()
+            queryStatistic.addCountStatistic(QueryStatisticsConstants.RESULT_SIZE, recordCount)
+            queryModel.getStatisticsRecorder.recordStatistics(queryStatistic)
+            // print executor query statistics for each task_id
+            queryModel.getStatisticsRecorder.logStatisticsAsTableExecutor()
           }
         }
         !finished
@@ -237,13 +241,18 @@ class CarbonScanRDD[V: ClassTag](
         if (queryModel.getLimit != -1 && recordCount >= queryModel.getLimit) {
           clearDictionaryCache(queryModel.getColumnToDictionaryMapping)
           if (null != queryModel.getStatisticsRecorder) {
-            val queryStatistic = new QueryStatistic
+            var queryStatistic = new QueryStatistic()
             queryStatistic
-              .addFixedTimeStatistic("Total Time taken to execute the query in executor Side",
+              .addFixedTimeStatistic(QueryStatisticsConstants.EXECUTOR_PART,
                 System.currentTimeMillis - queryStartTime
               )
-            queryModel.getStatisticsRecorder.recordStatistics(queryStatistic);
-            queryModel.getStatisticsRecorder.logStatistics();
+            queryModel.getStatisticsRecorder.recordStatistics(queryStatistic)
+            // result size
+            queryStatistic = new QueryStatistic()
+            queryStatistic.addCountStatistic(QueryStatisticsConstants.RESULT_SIZE, recordCount)
+            queryModel.getStatisticsRecorder.recordStatistics(queryStatistic)
+            // print executor query statistics for each task_id
+            queryModel.getStatisticsRecorder.logStatisticsAsTableExecutor()
           }
         }
         keyClass.getValue(rowIterator.next())

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index bdb2a69..028c52f 100644
--- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -163,4 +163,5 @@ object CarbonScalaUtil extends Logging {
     }
     kettleHomePath
   }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
index 3cfe301..972de05 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
@@ -34,8 +34,9 @@ import org.apache.spark.sql.hive._
 import org.apache.spark.sql.optimizer.CarbonOptimizer
 
 import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
-import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory}
 import org.apache.carbondata.spark.rdd.CarbonDataFrameRDD
 
 class CarbonContext(
@@ -67,12 +68,13 @@ class CarbonContext(
   override lazy val catalog = {
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.STORE_LOCATION, storePath)
-    new CarbonMetastoreCatalog(this, storePath, metadataHive) with OverrideCatalog
+    new CarbonMetastoreCatalog(this, storePath, metadataHive, queryId) with OverrideCatalog
   }
 
   @transient
   override protected[sql] lazy val analyzer =
     new Analyzer(catalog, functionRegistry, conf) {
+
       override val extendedResolutionRules =
         catalog.ParquetConversions ::
         catalog.CreateTables ::
@@ -118,15 +120,21 @@ class CarbonContext(
   @transient
   val LOGGER = LogServiceFactory.getLogService(CarbonContext.getClass.getName)
 
+  var queryId: String = ""
+
   override def sql(sql: String): DataFrame = {
     // queryId will be unique for each query, creting query detail holder
-    val queryId: String = System.nanoTime() + ""
+    queryId = System.nanoTime() + ""
     this.setConf("queryId", queryId)
 
     CarbonContext.updateCarbonPorpertiesPath(this)
     val sqlString = sql.toUpperCase
     LOGGER.info(s"Query [$sqlString]")
+    val recorder = CarbonTimeStatisticsFactory.getQueryStatisticsRecorderInstance()
+    val statistic = new QueryStatistic()
     val logicPlan: LogicalPlan = parseSql(sql)
+    statistic.addStatistics(QueryStatisticsConstants.SQL_PARSE, System.currentTimeMillis())
+    recorder.recordStatisticsForDriver(statistic, queryId)
     val result = new CarbonDataFrameRDD(this, logicPlan)
 
     // We force query optimization to happen right away instead of letting it happen lazily like
@@ -218,6 +226,7 @@ object CarbonContext {
   /**
    *
    * Requesting the extra executors other than the existing ones.
+ *
    * @param sc
    * @param numExecutors
    * @return

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index e4fb857..b29b2e8 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -32,6 +32,7 @@ import org.apache.carbondata.core.carbon.{AbsoluteTableIdentifier, ColumnIdentif
 import org.apache.carbondata.core.carbon.metadata.datatype.DataType
 import org.apache.carbondata.core.carbon.metadata.encoder.Encoding
 import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
+import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
 import org.apache.carbondata.core.util.DataTypeUtil
 
 /**
@@ -147,14 +148,17 @@ case class CarbonDictionaryDecoder(
 
   override def canProcessSafeRows: Boolean = true
 
+
+
   override def doExecute(): RDD[InternalRow] = {
     attachTree(this, "execute") {
       val storePath = sqlContext.catalog.asInstanceOf[CarbonMetastoreCatalog].storePath
+      val queryId = sqlContext.getConf("queryId", System.nanoTime() + "")
       val absoluteTableIdentifiers = relations.map { relation =>
         val carbonTable = relation.carbonRelation.carbonRelation.metaData.carbonTable
         (carbonTable.getFactTableName, carbonTable.getAbsoluteTableIdentifier)
       }.toMap
-
+      val recorder = new QueryStatisticsRecorder(queryId)
       if (isRequiredToDecode) {
         val dataTypes = child.output.map { attr => attr.dataType }
         child.execute().mapPartitions { iter =>
@@ -166,9 +170,21 @@ case class CarbonDictionaryDecoder(
           val dictIndex = dicts.zipWithIndex.filter(x => x._1 != null).map(x => x._2)
           new Iterator[InternalRow] {
             val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
-            override final def hasNext: Boolean = iter.hasNext
-
+            var flag = true
+            var total = 0L
+            override final def hasNext: Boolean = {
+              flag = iter.hasNext
+              if (false == flag && total > 0) {
+                val queryStatistic = new QueryStatistic()
+                queryStatistic
+                  .addFixedTimeStatistic(QueryStatisticsConstants.PREPARE_RESULT, total)
+                recorder.recordStatistics(queryStatistic)
+                recorder.logStatistics()
+              }
+              flag
+            }
             override final def next(): InternalRow = {
+              val startTime = System.currentTimeMillis()
               val row: InternalRow = iter.next()
               val data = row.toSeq(dataTypes).toArray
               dictIndex.foreach { index =>
@@ -178,7 +194,9 @@ case class CarbonDictionaryDecoder(
                       getDictionaryColumnIds(index)._3)
                 }
               }
-              unsafeProjection(new GenericMutableRow(data))
+              val result = unsafeProjection(new GenericMutableRow(data))
+              total += System.currentTimeMillis() - startTime
+              result
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
index 1452aa2..c2e85d1 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
@@ -55,7 +55,7 @@ case class CarbonScan(
 
     plan.setOutLocationPath(
       CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS))
-    plan.setQueryId(System.nanoTime() + "")
+    plan.setQueryId(ocRaw.getConf("queryId", System.nanoTime() + ""))
     processFilterExpressions(plan)
     plan
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/346a869c/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
index a9d6077..8db9800 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
@@ -40,12 +40,13 @@ import org.apache.carbondata.core.carbon.metadata.CarbonMetadata
 import org.apache.carbondata.core.carbon.metadata.converter.ThriftWrapperSchemaConverterImpl
 import org.apache.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.carbon.path.{CarbonStorePath, CarbonTablePath}
+import org.apache.carbondata.core.carbon.querystatistics.{QueryStatistic, QueryStatisticsConstants, QueryStatisticsRecorder}
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastorage.store.filesystem.CarbonFile
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory
 import org.apache.carbondata.core.datastorage.store.impl.FileFactory.FileType
 import org.apache.carbondata.core.reader.ThriftReader
-import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonTimeStatisticsFactory, CarbonUtil}
 import org.apache.carbondata.core.writer.ThriftWriter
 import org.apache.carbondata.format.{SchemaEvolutionEntry, TableInfo}
 import org.apache.carbondata.lcm.locks.ZookeeperInit
@@ -99,7 +100,7 @@ case class DictionaryMap(dictionaryMap: Map[String, Boolean]) {
 }
 
 class CarbonMetastoreCatalog(hiveContext: HiveContext, val storePath: String,
-    client: ClientInterface)
+    client: ClientInterface, queryId: String)
   extends HiveMetastoreCatalog(client, hiveContext)
     with spark.Logging {
 
@@ -151,7 +152,8 @@ class CarbonMetastoreCatalog(hiveContext: HiveContext, val storePath: String,
   }
 
   def loadMetadata(metadataPath: String): MetaData = {
-
+    val recorder = CarbonTimeStatisticsFactory.getQueryStatisticsRecorderInstance()
+    val statistic = new QueryStatistic()
     // creating zookeeper instance once.
     // if zookeeper is configured as carbon lock type.
     val zookeeperUrl: String = hiveContext.getConf(CarbonCommonConstants.ZOOKEEPER_URL, null)
@@ -176,8 +178,10 @@ class CarbonMetastoreCatalog(hiveContext: HiveContext, val storePath: String,
     val metaDataBuffer = new ArrayBuffer[TableMeta]
     fillMetaData(metadataPath, fileType, metaDataBuffer)
     updateSchemasUpdatedTime("", "")
+    statistic.addStatistics(QueryStatisticsConstants.LOAD_META,
+      System.currentTimeMillis())
+    recorder.recordStatisticsForDriver(statistic, queryId)
     MetaData(metaDataBuffer)
-
   }
 
   private def fillMetaData(basePath: String, fileType: FileType,